diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index fe01b92036377..009ebe90ddf51 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -154,15 +154,18 @@ jobs: python3.8 -m pip install numpy pyarrow pandas scipy python3.8 -m pip list # SparkR - - name: Install R 3.6 - uses: r-lib/actions/setup-r@v1 + - name: Install R 4.0 if: contains(matrix.modules, 'sparkr') - with: - r-version: 3.6 + run: | + sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" + curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev - name: Install R packages if: contains(matrix.modules, 'sparkr') run: | - sudo apt-get install -y libcurl4-openssl-dev + # qpdf is required to reduce the size of PDFs to make CRAN check pass. See SPARK-32497. + sudo apt-get install -y libcurl4-openssl-dev qpdf sudo Rscript -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'devtools', 'e1071', 'survival', 'arrow', 'roxygen2'), repos='https://cloud.r-project.org/')" # Show installed packages in R. sudo Rscript -e 'pkg_list <- as.data.frame(installed.packages()[, c(1,3:4)]); pkg_list[is.na(pkg_list$Priority), 1:2, drop = FALSE]' @@ -200,11 +203,15 @@ jobs: architecture: x64 - name: Install Python linter dependencies run: | - pip3 install flake8 sphinx numpy - - name: Install R 3.6 - uses: r-lib/actions/setup-r@v1 - with: - r-version: 3.6 + # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. + # See also https://github.com/sphinx-doc/sphinx/issues/7551. + pip3 install flake8 'sphinx<3.1.0' numpy pydata_sphinx_theme + - name: Install R 4.0 + run: | + sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" + curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev - name: Install R linter dependencies and SparkR run: | sudo apt-get install -y libcurl4-openssl-dev @@ -218,7 +225,9 @@ jobs: - name: Install dependencies for documentation generation run: | sudo apt-get install -y libcurl4-openssl-dev pandoc - pip install sphinx mkdocs numpy + # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. + # See also https://github.com/sphinx-doc/sphinx/issues/7551. + pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme gem install jekyll jekyll-redirect-from rouge sudo Rscript -e "install.packages(c('devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2'), repos='https://cloud.r-project.org/')" - name: Scala linter @@ -237,3 +246,28 @@ jobs: run: | cd docs jekyll build + + java11: + name: Java 11 build + runs-on: ubuntu-latest + steps: + - name: Checkout Spark repository + uses: actions/checkout@v2 + - name: Cache Maven local repository + uses: actions/cache@v2 + with: + path: ~/.m2/repository + key: java11-maven-${{ hashFiles('**/pom.xml') }} + restore-keys: | + java11-maven- + - name: Install Java 11 + uses: actions/setup-java@v1 + with: + java-version: 11 + - name: Build with Maven + run: | + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" + export MAVEN_CLI_OPTS="--no-transfer-progress" + mkdir -p ~/.m2 + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Djava.version=11 install + rm -rf ~/.m2/repository/org/apache/spark diff --git a/.gitignore b/.gitignore index 198fdee39be95..0d8addeb10e21 100644 --- a/.gitignore +++ b/.gitignore @@ -64,6 +64,7 @@ python/lib/pyspark.zip python/.eggs/ python/deps python/docs/_site/ +python/docs/source/reference/api/ python/test_coverage/coverage_data python/test_coverage/htmlcov python/pyspark/python diff --git a/LICENSE b/LICENSE index 6b169b1447f14..df6bed16f4471 100644 --- a/LICENSE +++ b/LICENSE @@ -222,14 +222,13 @@ external/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaRe Python Software Foundation License ---------------------------------- -pyspark/heapq3.py -python/docs/_static/copybutton.js +python/docs/source/_static/copybutton.js BSD 3-Clause ------------ python/lib/py4j-*-src.zip -python/pyspark/cloudpickle.py +python/pyspark/cloudpickle/*.py python/pyspark/join.py core/src/main/resources/org/apache/spark/ui/static/d3.min.js diff --git a/LICENSE-binary b/LICENSE-binary index b50da6be4e697..d363661b1cc7e 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -557,12 +557,6 @@ jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api org.glassfish.hk2.external:jakarta.inject -Python Software Foundation License ----------------------------------- - -pyspark/heapq3.py - - Public Domain ------------- diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index c5c08bd7a0636..2047f0d75ca18 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -23,7 +23,7 @@ Suggests: testthat, e1071, survival, - arrow (>= 0.15.1) + arrow (>= 1.0.0) Collate: 'schema.R' 'generics.R' diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 15b3ce2935427..4eca5bd23c3bb 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1233,13 +1233,8 @@ setMethod("collect", port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) output <- tryCatch({ doServerAuth(conn, authSecret) - arrowTable <- arrow::read_arrow(readRaw(conn)) - # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. - if (exists("as_tibble", envir = asNamespace("arrow"))) { - as.data.frame(arrow::as_tibble(arrowTable), stringsAsFactors = stringsAsFactors) - } else { - as.data.frame(arrowTable, stringsAsFactors = stringsAsFactors) - } + arrowTable <- arrow::read_ipc_stream(readRaw(conn)) + as.data.frame(arrowTable, stringsAsFactors = stringsAsFactors) }, finally = { close(conn) }) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index 3e7c456bd548d..5d22340fb62a0 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -233,24 +233,13 @@ readMultipleObjectsWithKeys <- function(inputCon) { readDeserializeInArrow <- function(inputCon) { if (requireNamespace("arrow", quietly = TRUE)) { - # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. - useAsTibble <- exists("as_tibble", envir = asNamespace("arrow")) - - # Currently, there looks no way to read batch by batch by socket connection in R side, # See ARROW-4512. Therefore, it reads the whole Arrow streaming-formatted binary at once # for now. dataLen <- readInt(inputCon) arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") batches <- arrow::RecordBatchStreamReader$create(arrowData)$batches() - - if (useAsTibble) { - as_tibble <- get("as_tibble", envir = asNamespace("arrow")) - # Read all groupped batches. Tibble -> data.frame is cheap. - lapply(batches, function(batch) as.data.frame(as_tibble(batch))) - } else { - lapply(batches, function(batch) as.data.frame(batch)) - } + lapply(batches, function(batch) as.data.frame(batch)) } else { stop("'arrow' package should be installed.") } diff --git a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R index 97972753a78fa..16d93763ff038 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R +++ b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R @@ -312,4 +312,22 @@ test_that("Arrow optimization - unsupported types", { }) }) +test_that("SPARK-32478: gapply() Arrow optimization - error message for schema mismatch", { + skip_if_not_installed("arrow") + df <- createDataFrame(list(list(a = 1L, b = "a"))) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + expect_error( + count(gapply(df, "a", function(key, group) { group }, structType("a int, b int"))), + "expected IntegerType, IntegerType, got IntegerType, StringType") + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + sparkR.session.stop() diff --git a/bin/find-spark-home b/bin/find-spark-home index 617dbaa4fff86..462b538b00a04 100755 --- a/bin/find-spark-home +++ b/bin/find-spark-home @@ -33,9 +33,9 @@ elif [ ! -f "$FIND_SPARK_HOME_PYTHON_SCRIPT" ]; then export SPARK_HOME="$(cd "$(dirname "$0")"/..; pwd)" else # We are pip installed, use the Python script to resolve a reasonable SPARK_HOME - # Default to standard python interpreter unless told otherwise + # Default to standard python3 interpreter unless told otherwise if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then - PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python"}" + PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python3"}" fi export SPARK_HOME=$($PYSPARK_DRIVER_PYTHON "$FIND_SPARK_HOME_PYTHON_SCRIPT") fi diff --git a/bin/find-spark-home.cmd b/bin/find-spark-home.cmd index 6025f67c38de4..f795d146d49c7 100644 --- a/bin/find-spark-home.cmd +++ b/bin/find-spark-home.cmd @@ -20,8 +20,8 @@ rem rem Path to Python script finding SPARK_HOME set FIND_SPARK_HOME_PYTHON_SCRIPT=%~dp0find_spark_home.py -rem Default to standard python interpreter unless told otherwise -set PYTHON_RUNNER=python +rem Default to standard python3 interpreter unless told otherwise +set PYTHON_RUNNER=python3 rem If PYSPARK_DRIVER_PYTHON is set, it overwrites the python version if not "x%PYSPARK_DRIVER_PYTHON%"=="x" ( set PYTHON_RUNNER=%PYSPARK_DRIVER_PYTHON% diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index 5f98cc34b6bab..fe725a4e1a368 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -21,42 +21,42 @@ rem This script loads spark-env.cmd if it exists, and ensures it is only loaded rem spark-env.cmd is loaded from SPARK_CONF_DIR if set, or within the current directory's rem conf\ subdirectory. -set SPARK_ENV_CMD=spark-env.cmd -if [%SPARK_ENV_LOADED%] == [] ( +if not defined SPARK_ENV_LOADED ( set SPARK_ENV_LOADED=1 if [%SPARK_CONF_DIR%] == [] ( set SPARK_CONF_DIR=%~dp0..\conf ) - set SPARK_ENV_CMD=%SPARK_CONF_DIR%\%SPARK_ENV_CMD% - if exist %SPARK_ENV_CMD% ( - call %SPARK_ENV_CMD% - ) + call :LoadSparkEnv ) rem Setting SPARK_SCALA_VERSION if not already set. -rem TODO: revisit for Scala 2.13 support -set SPARK_SCALA_VERSION=2.12 -rem if [%SPARK_SCALA_VERSION%] == [] ( -rem set SCALA_VERSION_1=2.12 -rem set SCALA_VERSION_2=2.11 -rem -rem set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% -rem set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% -rem set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables -rem if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( -rem echo "Presence of build for multiple Scala versions detected (%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%)." -rem echo "Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in %SPARK_ENV_CMD%." -rem echo "Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd." -rem echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." -rem exit 1 -rem ) -rem if exist %ASSEMBLY_DIR1% ( -rem set SPARK_SCALA_VERSION=%SCALA_VERSION_1% -rem ) else ( -rem set SPARK_SCALA_VERSION=%SCALA_VERSION_2% -rem ) -rem ) +set SCALA_VERSION_1=2.13 +set SCALA_VERSION_2=2.12 + +set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% +set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% +set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables + +if not defined SPARK_SCALA_VERSION ( + if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( + echo Presence of build for multiple Scala versions detected ^(%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%^). + echo Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in spark-env.cmd. + echo Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd. + echo Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd. + exit 1 + ) + if exist %ASSEMBLY_DIR1% ( + set SPARK_SCALA_VERSION=%SCALA_VERSION_1% + ) else ( + set SPARK_SCALA_VERSION=%SCALA_VERSION_2% + ) +) exit /b 0 + +:LoadSparkEnv +if exist "%SPARK_CONF_DIR%\spark-env.cmd" ( + call "%SPARK_CONF_DIR%\spark-env.cmd" +) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 107e7991c28bc..04adaeed7ac61 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -43,25 +43,23 @@ fi # Setting SPARK_SCALA_VERSION if not already set. -# TODO: revisit for Scala 2.13 support -export SPARK_SCALA_VERSION=2.12 -#if [ -z "$SPARK_SCALA_VERSION" ]; then -# SCALA_VERSION_1=2.12 -# SCALA_VERSION_2=2.11 -# -# ASSEMBLY_DIR_1="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_1}" -# ASSEMBLY_DIR_2="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_2}" -# ENV_VARIABLE_DOC="https://spark.apache.org/docs/latest/configuration.html#environment-variables" -# if [[ -d "$ASSEMBLY_DIR_1" && -d "$ASSEMBLY_DIR_2" ]]; then -# echo "Presence of build for multiple Scala versions detected ($ASSEMBLY_DIR_1 and $ASSEMBLY_DIR_2)." 1>&2 -# echo "Remove one of them or, export SPARK_SCALA_VERSION=$SCALA_VERSION_1 in ${SPARK_ENV_SH}." 1>&2 -# echo "Visit ${ENV_VARIABLE_DOC} for more details about setting environment variables in spark-env.sh." 1>&2 -# exit 1 -# fi -# -# if [[ -d "$ASSEMBLY_DIR_1" ]]; then -# export SPARK_SCALA_VERSION=${SCALA_VERSION_1} -# else -# export SPARK_SCALA_VERSION=${SCALA_VERSION_2} -# fi -#fi +if [ -z "$SPARK_SCALA_VERSION" ]; then + SCALA_VERSION_1=2.13 + SCALA_VERSION_2=2.12 + + ASSEMBLY_DIR_1="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_1}" + ASSEMBLY_DIR_2="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_2}" + ENV_VARIABLE_DOC="https://spark.apache.org/docs/latest/configuration.html#environment-variables" + if [[ -d "$ASSEMBLY_DIR_1" && -d "$ASSEMBLY_DIR_2" ]]; then + echo "Presence of build for multiple Scala versions detected ($ASSEMBLY_DIR_1 and $ASSEMBLY_DIR_2)." 1>&2 + echo "Remove one of them or, export SPARK_SCALA_VERSION=$SCALA_VERSION_1 in ${SPARK_ENV_SH}." 1>&2 + echo "Visit ${ENV_VARIABLE_DOC} for more details about setting environment variables in spark-env.sh." 1>&2 + exit 1 + fi + + if [[ -d "$ASSEMBLY_DIR_1" ]]; then + export SPARK_SCALA_VERSION=${SCALA_VERSION_1} + else + export SPARK_SCALA_VERSION=${SCALA_VERSION_2} + fi +fi diff --git a/bin/pyspark b/bin/pyspark index ad4132fb59eb0..463a2dcfc7e6c 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -37,9 +37,9 @@ if [[ -n "$IPYTHON" || -n "$IPYTHON_OPTS" ]]; then exit 1 fi -# Default to standard python interpreter unless told otherwise +# Default to standard python3 interpreter unless told otherwise if [[ -z "$PYSPARK_PYTHON" ]]; then - PYSPARK_PYTHON=python + PYSPARK_PYTHON=python3 fi if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then PYSPARK_DRIVER_PYTHON=$PYSPARK_PYTHON diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java index 98f33b70fea23..121dfbd4f6838 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java @@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; @@ -153,24 +154,72 @@ public void write(Object value) throws Exception { try (WriteBatch batch = db().createWriteBatch()) { byte[] data = serializer.serialize(value); synchronized (ti) { - Object existing; - try { - existing = get(ti.naturalIndex().entityKey(null, value), value.getClass()); - } catch (NoSuchElementException e) { - existing = null; - } + updateBatch(batch, value, data, value.getClass(), ti.naturalIndex(), ti.indices()); + db().write(batch); + } + } + } + + public void writeAll(List values) throws Exception { + Preconditions.checkArgument(values != null && !values.isEmpty(), + "Non-empty values required."); + + // Group by class, in case there are values from different classes in the values + // Typical usecase is for this to be a single class. + // A NullPointerException will be thrown if values contain null object. + for (Map.Entry, ? extends List> entry : + values.stream().collect(Collectors.groupingBy(Object::getClass)).entrySet()) { + + final Iterator valueIter = entry.getValue().iterator(); + final Iterator serializedValueIter; + + // Deserialize outside synchronized block + List list = new ArrayList<>(entry.getValue().size()); + for (Object value : values) { + list.add(serializer.serialize(value)); + } + serializedValueIter = list.iterator(); + + final Class klass = entry.getKey(); + final LevelDBTypeInfo ti = getTypeInfo(klass); - PrefixCache cache = new PrefixCache(value); - byte[] naturalKey = ti.naturalIndex().toKey(ti.naturalIndex().getValue(value)); - for (LevelDBTypeInfo.Index idx : ti.indices()) { - byte[] prefix = cache.getPrefix(idx); - idx.add(batch, value, existing, data, naturalKey, prefix); + synchronized (ti) { + final LevelDBTypeInfo.Index naturalIndex = ti.naturalIndex(); + final Collection indices = ti.indices(); + + try (WriteBatch batch = db().createWriteBatch()) { + while (valueIter.hasNext()) { + updateBatch(batch, valueIter.next(), serializedValueIter.next(), klass, + naturalIndex, indices); + } + db().write(batch); } - db().write(batch); } } } + private void updateBatch( + WriteBatch batch, + Object value, + byte[] data, + Class klass, + LevelDBTypeInfo.Index naturalIndex, + Collection indices) throws Exception { + Object existing; + try { + existing = get(naturalIndex.entityKey(null, value), klass); + } catch (NoSuchElementException e) { + existing = null; + } + + PrefixCache cache = new PrefixCache(value); + byte[] naturalKey = naturalIndex.toKey(naturalIndex.getValue(value)); + for (LevelDBTypeInfo.Index idx : indices) { + byte[] prefix = cache.getPrefix(idx); + idx.add(batch, value, existing, data, naturalKey, prefix); + } + } + @Override public void delete(Class type, Object naturalKey) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala index ff728f0ebcb85..4c535a8dd0411 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.util.sketch import scala.util.Random -import org.scalatest.FunSuite // scalastyle:ignore funsuite +import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -class BitArraySuite extends FunSuite { // scalastyle:ignore funsuite +class BitArraySuite extends AnyFunSuite { // scalastyle:ignore funsuite test("error case when create BitArray") { intercept[IllegalArgumentException](new BitArray(0)) diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala index a0408d2da4dff..8b289fc86af0f 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala @@ -22,9 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.reflect.ClassTag import scala.util.Random -import org.scalatest.FunSuite // scalastyle:ignore funsuite +import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -class BloomFilterSuite extends FunSuite { // scalastyle:ignore funsuite +class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite private final val EPSILON = 0.01 // Serializes and deserializes a given `BloomFilter`, then checks whether the deserialized diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala index 174eb01986c4f..087dae26047ef 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala @@ -22,9 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.reflect.ClassTag import scala.util.Random -import org.scalatest.FunSuite // scalastyle:ignore funsuite +import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -class CountMinSketchSuite extends FunSuite { // scalastyle:ignore funsuite +class CountMinSketchSuite extends AnyFunSuite { // scalastyle:ignore funsuite private val epsOfTotalCount = 0.01 private val confidence = 0.9 diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala index 72aa682bb95bc..69a082053aa65 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala @@ -21,14 +21,15 @@ import org.apache.commons.text.similarity.LevenshteinDistance import org.scalacheck.{Arbitrary, Gen} import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks // scalastyle:off -import org.scalatest.{FunSuite, Matchers} +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.must.Matchers import org.apache.spark.unsafe.types.UTF8String.{fromString => toUTF8} /** * This TestSuite utilize ScalaCheck to generate randomized inputs for UTF8String testing. */ -class UTF8StringPropertyCheckSuite extends FunSuite with ScalaCheckDrivenPropertyChecks with Matchers { +class UTF8StringPropertyCheckSuite extends AnyFunSuite with ScalaCheckDrivenPropertyChecks with Matchers { // scalastyle:on test("toString") { diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt index db23cf5c12ea7..4b16bd445f39d 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt @@ -1,64 +1,64 @@ -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 170 178 9 1.2 849.7 1.0X -Deserialization 530 535 9 0.4 2651.1 0.3X +------------------------------------------------------------------------------------------------------------------------- +Serialization 175 183 12 1.1 874.1 1.0X +Deserialization 458 462 6 0.4 2288.6 0.4X -Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized MapStatus sizes: 410 bytes Compressed Serialized Broadcast MapStatus sizes: 2 MB -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 157 165 7 1.3 785.4 1.0X -Deserialization 495 588 79 0.4 2476.7 0.3X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 160 171 8 1.2 801.1 1.0X +Deserialization 453 484 38 0.4 2263.4 0.4X Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 344 351 4 0.6 1720.4 1.0X -Deserialization 527 579 99 0.4 2635.9 0.7X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 343 346 2 0.6 1717.1 1.0X +Deserialization 492 540 59 0.4 2459.6 0.7X -Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized MapStatus sizes: 426 bytes Compressed Serialized Broadcast MapStatus sizes: 13 MB -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 317 321 4 0.6 1583.8 1.0X -Deserialization 530 540 15 0.4 2648.3 0.6X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 297 299 2 0.7 1486.2 1.0X +Deserialization 489 535 87 0.4 2446.5 0.6X Compressed Serialized MapStatus sizes: 13 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1738 1849 156 0.1 8692.0 1.0X -Deserialization 946 977 33 0.2 4730.2 1.8X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 1641 1819 252 0.1 8204.1 1.0X +Deserialization 844 882 37 0.2 4219.7 1.9X -Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized MapStatus sizes: 553 bytes Compressed Serialized Broadcast MapStatus sizes: 121 MB -OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.8+10-post-Ubuntu-0ubuntu118.04.1 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1379 1432 76 0.1 6892.6 1.0X -Deserialization 929 941 19 0.2 4645.5 1.5X +---------------------------------------------------------------------------------------------------------------------------- +Serialization 1360 1412 73 0.1 6799.3 1.0X +Deserialization 850 859 13 0.2 4249.9 1.6X Compressed Serialized MapStatus sizes: 121 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt index 053f4bf771923..996b79b896dba 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt @@ -1,64 +1,64 @@ -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 178 187 15 1.1 887.5 1.0X -Deserialization 530 558 32 0.4 2647.5 0.3X +------------------------------------------------------------------------------------------------------------------------- +Serialization 193 202 15 1.0 966.2 1.0X +Deserialization 477 521 44 0.4 2387.5 0.4X -Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized MapStatus sizes: 410 bytes Compressed Serialized Broadcast MapStatus sizes: 2 MB -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 167 175 7 1.2 835.7 1.0X -Deserialization 523 537 22 0.4 2616.2 0.3X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 181 187 11 1.1 907.1 1.0X +Deserialization 477 492 27 0.4 2383.5 0.4X Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 351 416 147 0.6 1754.4 1.0X -Deserialization 546 551 8 0.4 2727.6 0.6X +-------------------------------------------------------------------------------------------------------------------------- +Serialization 359 436 162 0.6 1797.5 1.0X +Deserialization 506 514 15 0.4 2530.8 0.7X -Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized MapStatus sizes: 426 bytes Compressed Serialized Broadcast MapStatus sizes: 13 MB -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 320 321 1 0.6 1598.0 1.0X -Deserialization 542 549 7 0.4 2709.0 0.6X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 322 323 0 0.6 1610.5 1.0X +Deserialization 508 513 5 0.4 2538.4 0.6X Compressed Serialized MapStatus sizes: 13 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1671 1877 290 0.1 8357.3 1.0X -Deserialization 943 970 32 0.2 4715.8 1.8X +--------------------------------------------------------------------------------------------------------------------------- +Serialization 1740 1903 231 0.1 8700.0 1.0X +Deserialization 872 888 24 0.2 4360.9 2.0X -Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized MapStatus sizes: 553 bytes Compressed Serialized Broadcast MapStatus sizes: 121 MB -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 1373 1436 89 0.1 6865.0 1.0X -Deserialization 940 970 37 0.2 4699.1 1.5X +---------------------------------------------------------------------------------------------------------------------------- +Serialization 1461 1469 11 0.1 7306.1 1.0X +Deserialization 871 889 22 0.2 4353.9 1.7X Compressed Serialized MapStatus sizes: 121 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 579e7ff320f5c..c0e72b57d48bd 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -162,6 +162,16 @@ public void onSpeculativeTaskSubmitted(SparkListenerSpeculativeTaskSubmitted spe onEvent(speculativeTask); } + public void onUnschedulableTaskSetAdded( + SparkListenerUnschedulableTaskSetAdded unschedulableTaskSetAdded) { + onEvent(unschedulableTaskSetAdded); + } + + public void onUnschedulableTaskSetRemoved( + SparkListenerUnschedulableTaskSetRemoved unschedulableTaskSetRemoved) { + onEvent(unschedulableTaskSetRemoved); + } + @Override public void onResourceProfileAdded(SparkListenerResourceProfileAdded event) { onEvent(event); diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 620a6fe2f9d72..85409d599ccaa 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -281,6 +281,7 @@ private[spark] class ExecutorAllocationManager( private def maxNumExecutorsNeededPerResourceProfile(rpId: Int): Int = { val pending = listener.totalPendingTasksPerResourceProfile(rpId) val pendingSpeculative = listener.pendingSpeculativeTasksPerResourceProfile(rpId) + val unschedulableTaskSets = listener.pendingUnschedulableTaskSetsPerResourceProfile(rpId) val running = listener.totalRunningTasksPerResourceProfile(rpId) val numRunningOrPendingTasks = pending + running val rp = resourceProfileManager.resourceProfileFromId(rpId) @@ -289,13 +290,27 @@ private[spark] class ExecutorAllocationManager( s" tasksperexecutor: $tasksPerExecutor") val maxNeeded = math.ceil(numRunningOrPendingTasks * executorAllocationRatio / tasksPerExecutor).toInt - if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) { + + val maxNeededWithSpeculationLocalityOffset = + if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) { // If we have pending speculative tasks and only need a single executor, allocate one more // to satisfy the locality requirements of speculation maxNeeded + 1 } else { maxNeeded } + + if (unschedulableTaskSets > 0) { + // Request additional executors to account for task sets having tasks that are unschedulable + // due to blacklisting when the active executor count has already reached the max needed + // which we would normally get. + val maxNeededForUnschedulables = math.ceil(unschedulableTaskSets * executorAllocationRatio / + tasksPerExecutor).toInt + math.max(maxNeededWithSpeculationLocalityOffset, + executorMonitor.executorCountWithResourceProfile(rpId) + maxNeededForUnschedulables) + } else { + maxNeededWithSpeculationLocalityOffset + } } private def totalRunningTasksPerResourceProfile(id: Int): Int = synchronized { @@ -622,6 +637,12 @@ private[spark] class ExecutorAllocationManager( private val resourceProfileIdToStageAttempt = new mutable.HashMap[Int, mutable.Set[StageAttempt]] + // Keep track of unschedulable task sets due to blacklisting. This is a Set of StageAttempt's + // because we'll only take the last unschedulable task in a taskset although there can be more. + // This is done in order to avoid costly loops in the scheduling. + // Check TaskSetManager#getCompletelyBlacklistedTaskIfAny for more details. + private val unschedulableTaskSets = new mutable.HashSet[StageAttempt] + // stageAttempt to tuple (the number of task with locality preferences, a map where each pair // is a node and the number of tasks that would like to be scheduled on that node, and // the resource profile id) map, @@ -789,6 +810,28 @@ private[spark] class ExecutorAllocationManager( } } + override def onUnschedulableTaskSetAdded( + unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded): Unit = { + val stageId = unschedulableTaskSetAdded.stageId + val stageAttemptId = unschedulableTaskSetAdded.stageAttemptId + val stageAttempt = StageAttempt(stageId, stageAttemptId) + allocationManager.synchronized { + unschedulableTaskSets.add(stageAttempt) + allocationManager.onSchedulerBacklogged() + } + } + + override def onUnschedulableTaskSetRemoved( + unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved): Unit = { + val stageId = unschedulableTaskSetRemoved.stageId + val stageAttemptId = unschedulableTaskSetRemoved.stageAttemptId + val stageAttempt = StageAttempt(stageId, stageAttemptId) + allocationManager.synchronized { + // Clear unschedulableTaskSets since atleast one task becomes schedulable now + unschedulableTaskSets.remove(stageAttempt) + } + } + /** * An estimate of the total number of pending tasks remaining for currently running stages. Does * not account for tasks which may have failed and been resubmitted. @@ -829,6 +872,16 @@ private[spark] class ExecutorAllocationManager( numTotalTasks - numRunning } + /** + * Currently we only know when a task set has an unschedulable task, we don't know + * the exact number and since the allocation manager isn't tied closely with the scheduler, + * we use the number of tasks sets that are unschedulable as a heuristic to add more executors. + */ + def pendingUnschedulableTaskSetsPerResourceProfile(rp: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq + attempts.filter(attempt => unschedulableTaskSets.contains(attempt)).size + } + def hasPendingTasks: Boolean = { hasPendingSpeculativeTasks || hasPendingRegularTasks } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 32251df6f4bbe..64102ccc05882 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -49,7 +49,7 @@ import org.apache.spark.util._ * * All public methods of this class are thread-safe. */ -private class ShuffleStatus(numPartitions: Int) { +private class ShuffleStatus(numPartitions: Int) extends Logging { private val (readLock, writeLock) = { val lock = new ReentrantReadWriteLock() @@ -121,12 +121,28 @@ private class ShuffleStatus(numPartitions: Int) { mapStatuses(mapIndex) = status } + /** + * Update the map output location (e.g. during migration). + */ + def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit = withWriteLock { + val mapStatusOpt = mapStatuses.find(_.mapId == mapId) + mapStatusOpt match { + case Some(mapStatus) => + logInfo(s"Updating map output for ${mapId} to ${bmAddress}") + mapStatus.updateLocation(bmAddress) + invalidateSerializedMapOutputStatusCache() + case None => + logError(s"Asked to update map output ${mapId} for untracked map status.") + } + } + /** * Remove the map output which was served by the specified block manager. * This is a no-op if there is no registered map output or if the registered output is from a * different block manager. */ def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = withWriteLock { + logDebug(s"Removing existing map output ${mapIndex} ${bmAddress}") if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) { _numAvailableOutputs -= 1 mapStatuses(mapIndex) = null @@ -139,6 +155,7 @@ private class ShuffleStatus(numPartitions: Int) { * outputs which are served by an external shuffle server (if one exists). */ def removeOutputsOnHost(host: String): Unit = withWriteLock { + logDebug(s"Removing outputs for host ${host}") removeOutputsByFilter(x => x.host == host) } @@ -148,6 +165,7 @@ private class ShuffleStatus(numPartitions: Int) { * still registered with that execId. */ def removeOutputsOnExecutor(execId: String): Unit = withWriteLock { + logDebug(s"Removing outputs for execId ${execId}") removeOutputsByFilter(x => x.executorId == execId) } @@ -265,7 +283,7 @@ private[spark] class MapOutputTrackerMasterEndpoint( override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = context.senderAddress.hostPort - logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) + logInfo(s"Asked to send map output locations for shuffle ${shuffleId} to ${hostPort}") tracker.post(new GetMapOutputMessage(shuffleId, context)) case StopMapOutputTracker => @@ -465,6 +483,15 @@ private[spark] class MapOutputTrackerMaster( } } + def updateMapOutput(shuffleId: Int, mapId: Long, bmAddress: BlockManagerId): Unit = { + shuffleStatuses.get(shuffleId) match { + case Some(shuffleStatus) => + shuffleStatus.updateMapOutput(mapId, bmAddress) + case None => + logError(s"Asked to update map output for unknown shuffle ${shuffleId}") + } + } + def registerMapOutput(shuffleId: Int, mapIndex: Int, status: MapStatus): Unit = { shuffleStatuses(shuffleId).addMapOutput(mapIndex, status) } @@ -745,7 +772,12 @@ private[spark] class MapOutputTrackerMaster( override def stop(): Unit = { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() - sendTracker(StopMapOutputTracker) + try { + sendTracker(StopMapOutputTracker) + } catch { + case e: SparkException => + logError("Could not tell tracker we are stopping.", e) + } trackerEndpoint = null shuffleStatuses.clear() } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 06abc0541a9a9..5e0eaa478547c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -83,8 +83,10 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() - // In order to prevent SparkContext from being created in executors. - SparkContext.assertOnDriver() + if (!config.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { + // In order to prevent SparkContext from being created in executors. + SparkContext.assertOnDriver() + } // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having started construction. @@ -1601,7 +1603,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Get the max number of tasks that can be concurrent launched based on the ResourceProfile - * being used. + * could be used, even if some of them are being used at the moment. * Note that please don't cache the value returned by this method, because the number can change * due to add/remove executors. * diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8ba1739831803..d543359f4dedf 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -367,7 +367,8 @@ object SparkEnv extends Logging { externalShuffleClient } else { None - }, blockManagerInfo)), + }, blockManagerInfo, + mapOutputTracker.asInstanceOf[MapOutputTrackerMaster])), registerOrLookupEndpoint( BlockManagerMaster.DRIVER_HEARTBEAT_ENDPOINT_NAME, new BlockManagerMasterHeartbeatEndpoint(rpcEnv, isLocal, blockManagerInfo)), diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index d459627930f4c..6947d1c72f12b 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -179,11 +179,20 @@ private[spark] object TestUtils { destDir: File, toStringValue: String = "", baseClass: String = null, - classpathUrls: Seq[URL] = Seq.empty): File = { + classpathUrls: Seq[URL] = Seq.empty, + implementsClasses: Seq[String] = Seq.empty, + extraCodeBody: String = ""): File = { val extendsText = Option(baseClass).map { c => s" extends ${c}" }.getOrElse("") + val implementsText = + "implements " + (implementsClasses :+ "java.io.Serializable").mkString(", ") val sourceFile = new JavaSourceFromString(className, - "public class " + className + extendsText + " implements java.io.Serializable {" + - " @Override public String toString() { return \"" + toStringValue + "\"; }}") + s""" + |public class $className $extendsText $implementsText { + | @Override public String toString() { return "$toStringValue"; } + | + | $extraCodeBody + |} + """.stripMargin) createCompiledClass(className, destDir, sourceFile, classpathUrls) } @@ -236,7 +245,13 @@ private[spark] object TestUtils { * Test if a command is available. */ def testCommandAvailable(command: String): Boolean = { - val attempt = Try(Process(command).run(ProcessLogger(_ => ())).exitValue()) + val attempt = if (Utils.isWindows) { + Try(Process(Seq( + "cmd.exe", "/C", s"where $command")).run(ProcessLogger(_ => ())).exitValue()) + } else { + Try(Process(Seq( + "sh", "-c", s"command -v $command")).run(ProcessLogger(_ => ())).exitValue()) + } attempt.isSuccess && attempt.get == 0 } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 39eb1ee731d50..608158caacd37 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -727,7 +727,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { * @note This does not necessarily mean the caching or computation was successful. */ def getPersistentRDDs: JMap[java.lang.Integer, JavaRDD[_]] = { - sc.getPersistentRDDs.mapValues(s => JavaRDD.fromRDD(s)) + sc.getPersistentRDDs.mapValues(s => JavaRDD.fromRDD(s)).toMap .asJava.asInstanceOf[JMap[java.lang.Integer, JavaRDD[_]]] } diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 18305ad3746a6..b7a64d75a8d47 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -108,6 +108,13 @@ private[deploy] object DeployMessages { case class Heartbeat(workerId: String, worker: RpcEndpointRef) extends DeployMessage + /** + * Used by the MasterWebUI to request the master to decommission all workers that are active on + * any of the given hostnames. + * @param hostnames: A list of hostnames without the ports. Like "localhost", "foo.bar.com" etc + */ + case class DecommissionWorkersOnHosts(hostnames: Seq[String]) + // Master to Worker sealed trait RegisterWorkerResponse @@ -158,8 +165,6 @@ private[deploy] object DeployMessages { case object ReregisterWithMaster // used when a worker attempts to reconnect to a master - case object DecommissionSelf // Mark as decommissioned. May be Master to Worker in the future. - // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription, driver: RpcEndpointRef) diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index fc849d7f4372f..33851d9145d0a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -76,8 +76,8 @@ class LocalSparkCluster( logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected workerRpcEnvs.foreach(_.shutdown()) - masterRpcEnvs.foreach(_.shutdown()) workerRpcEnvs.foreach(_.awaitTermination()) + masterRpcEnvs.foreach(_.shutdown()) masterRpcEnvs.foreach(_.awaitTermination()) masterRpcEnvs.clear() workerRpcEnvs.clear() diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index eedf5e969e291..a6da8393bf405 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -31,6 +31,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging import org.apache.spark.rpc._ +import org.apache.spark.scheduler.ExecutorDecommissionInfo import org.apache.spark.util.{RpcUtils, ThreadUtils} /** @@ -181,7 +182,8 @@ private[spark] class StandaloneAppClient( if (ExecutorState.isFinished(state)) { listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, workerLost) } else if (state == ExecutorState.DECOMMISSIONED) { - listener.executorDecommissioned(fullId, message.getOrElse("")) + listener.executorDecommissioned(fullId, + ExecutorDecommissionInfo(message.getOrElse(""), isHostDecommissioned = workerLost)) } case WorkerRemoved(id, host, message) => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala index 2e38a6847891d..e72f7e976bb0a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.client +import org.apache.spark.scheduler.ExecutorDecommissionInfo + /** * Callbacks invoked by deploy client when various events happen. There are currently five events: * connecting to the cluster, disconnecting, being given an executor, having an executor removed @@ -39,7 +41,7 @@ private[spark] trait StandaloneAppClientListener { def executorRemoved( fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit - def executorDecommissioned(fullId: String, message: String): Unit + def executorDecommissioned(fullId: String, decommissionInfo: ExecutorDecommissionInfo): Unit def workerRemoved(workerId: String, host: String, message: String): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index a73a5e9463204..bc3e53c4122e7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -530,10 +530,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // If the file is currently not being tracked by the SHS, add an entry for it and try // to parse it. This will allow the cleaner code to detect the file as stale later on // if it was not possible to parse it. - listing.write(LogInfo(reader.rootPath.toString(), newLastScanTime, LogType.EventLogs, - None, None, reader.fileSizeForLastIndex, reader.lastIndex, None, - reader.completed)) - reader.fileSizeForLastIndex > 0 + try { + listing.write(LogInfo(reader.rootPath.toString(), newLastScanTime, + LogType.EventLogs, None, None, reader.fileSizeForLastIndex, reader.lastIndex, + None, reader.completed)) + reader.fileSizeForLastIndex > 0 + } catch { + case _: FileNotFoundException => false + } + + case _: FileNotFoundException => + false } } .sortWith { case (entry1, entry2) => diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala index 96db86f8e745a..08db2bd0766c3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HybridStore.scala @@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ +import com.google.common.collect.Lists; + import org.apache.spark.util.kvstore._ /** @@ -144,10 +146,9 @@ private[history] class HybridStore extends KVStore { backgroundThread = new Thread(() => { try { for (klass <- klassMap.keys().asScala) { - val it = inMemoryStore.view(klass).closeableIterator() - while (it.hasNext()) { - levelDB.write(it.next()) - } + val values = Lists.newArrayList( + inMemoryStore.view(klass).closeableIterator()) + levelDB.writeAll(values) } listener.onSwitchToLevelDBSuccess() shouldUseInMemoryStore.set(false) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index d2e65db970380..220e1c963d5ea 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -22,7 +22,9 @@ import java.util.{Date, Locale} import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable import scala.util.Random +import scala.util.control.NonFatal import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} @@ -525,6 +527,13 @@ private[deploy] class Master( case KillExecutors(appId, executorIds) => val formattedExecutorIds = formatExecutorIds(executorIds) context.reply(handleKillExecutors(appId, formattedExecutorIds)) + + case DecommissionWorkersOnHosts(hostnames) => + if (state != RecoveryState.STANDBY) { + context.reply(decommissionWorkersOnHosts(hostnames)) + } else { + context.reply(0) + } } override def onDisconnected(address: RpcAddress): Unit = { @@ -863,6 +872,34 @@ private[deploy] class Master( true } + /** + * Decommission all workers that are active on any of the given hostnames. The decommissioning is + * asynchronously done by enqueueing WorkerDecommission messages to self. No checks are done about + * the prior state of the worker. So an already decommissioned worker will match as well. + * + * @param hostnames: A list of hostnames without the ports. Like "localhost", "foo.bar.com" etc + * + * Returns the number of workers that matched the hostnames. + */ + private def decommissionWorkersOnHosts(hostnames: Seq[String]): Integer = { + val hostnamesSet = hostnames.map(_.toLowerCase(Locale.ROOT)).toSet + val workersToRemove = addressToWorker + .filterKeys(addr => hostnamesSet.contains(addr.host.toLowerCase(Locale.ROOT))) + .values + + val workersToRemoveHostPorts = workersToRemove.map(_.hostPort) + logInfo(s"Decommissioning the workers with host:ports ${workersToRemoveHostPorts}") + + // The workers are removed async to avoid blocking the receive loop for the entire batch + workersToRemove.foreach(wi => { + logInfo(s"Sending the worker decommission to ${wi.id} and ${wi.endpoint}") + self.send(WorkerDecommission(wi.id, wi.endpoint)) + }) + + // Return the count of workers actually removed + workersToRemove.size + } + private def decommissionWorker(worker: WorkerInfo): Unit = { if (worker.state != WorkerState.DECOMMISSIONED) { logInfo("Decommissioning worker %s on %s:%d".format(worker.id, worker.host, worker.port)) @@ -871,7 +908,10 @@ private[deploy] class Master( logInfo("Telling app of decommission executors") exec.application.driver.send(ExecutorUpdated( exec.id, ExecutorState.DECOMMISSIONED, - Some("worker decommissioned"), None, workerLost = false)) + Some("worker decommissioned"), None, + // workerLost is being set to true here to let the driver know that the host (aka. worker) + // is also being decommissioned. + workerLost = true)) exec.state = ExecutorState.DECOMMISSIONED exec.application.removeExecutor(exec) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 86554ec4ec1c9..035f9d379471c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,9 +17,14 @@ package org.apache.spark.deploy.master.ui -import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import java.net.{InetAddress, NetworkInterface, SocketException} +import java.util.Locale +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} + +import org.apache.spark.deploy.DeployMessages.{DecommissionWorkersOnHosts, MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.MASTER_UI_DECOMMISSION_ALLOW_MODE import org.apache.spark.internal.config.UI.UI_KILL_ENABLED import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -36,6 +41,7 @@ class MasterWebUI( val masterEndpointRef = master.self val killEnabled = master.conf.get(UI_KILL_ENABLED) + val decommissionAllowMode = master.conf.get(MASTER_UI_DECOMMISSION_ALLOW_MODE) initialize() @@ -49,6 +55,27 @@ class MasterWebUI( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST"))) + attachHandler(createServletHandler("/workers/kill", new HttpServlet { + override def doPost(req: HttpServletRequest, resp: HttpServletResponse): Unit = { + val hostnames: Seq[String] = Option(req.getParameterValues("host")) + .getOrElse(Array[String]()).toSeq + if (!isDecommissioningRequestAllowed(req)) { + resp.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } else { + val removedWorkers = masterEndpointRef.askSync[Integer]( + DecommissionWorkersOnHosts(hostnames)) + logInfo(s"Decommissioning of hosts $hostnames decommissioned $removedWorkers workers") + if (removedWorkers > 0) { + resp.setStatus(HttpServletResponse.SC_OK) + } else if (removedWorkers == 0) { + resp.sendError(HttpServletResponse.SC_NOT_FOUND) + } else { + // We shouldn't even see this case. + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + } + } + } + }, "")) } def addProxy(): Unit = { @@ -64,6 +91,25 @@ class MasterWebUI( maybeWorkerUiAddress.orElse(maybeAppUiAddress) } + private def isLocal(address: InetAddress): Boolean = { + if (address.isAnyLocalAddress || address.isLoopbackAddress) { + return true + } + try { + NetworkInterface.getByInetAddress(address) != null + } catch { + case _: SocketException => false + } + } + + private def isDecommissioningRequestAllowed(req: HttpServletRequest): Boolean = { + decommissionAllowMode match { + case "ALLOW" => true + case "LOCAL" => isLocal(InetAddress.getByName(req.getRemoteAddr)) + case _ => false + } + } + } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index 4e91e72361488..a46864e2d3c9c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -139,7 +139,9 @@ private[deploy] object HadoopFSDelegationTokenProvider { def hadoopFSsToAccess( sparkConf: SparkConf, hadoopConf: Configuration): Set[FileSystem] = { + // scalastyle:off FileSystemGet val defaultFS = FileSystem.get(hadoopConf) + // scalastyle:on FileSystemGet val filesystemsToAccess = sparkConf.get(KERBEROS_FILESYSTEMS_TO_ACCESS) .map(new Path(_).getFileSystem(hadoopConf)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index aa8c46fc68315..862e685c2dce6 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -668,7 +668,7 @@ private[deploy] class Worker( finishedApps += id maybeCleanupApplication(id) - case DecommissionSelf => + case WorkerDecommission(_, _) => decommissionSelf() } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e072d7919450e..55fb76b3572a3 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -40,7 +40,7 @@ import org.apache.spark.resource.ResourceProfile import org.apache.spark.resource.ResourceProfile._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc._ -import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} +import org.apache.spark.scheduler.{ExecutorDecommissionInfo, ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, SignalUtils, ThreadUtils, Utils} @@ -64,7 +64,6 @@ private[spark] class CoarseGrainedExecutorBackend( private[this] val stopping = new AtomicBoolean(false) var executor: Executor = null - @volatile private var decommissioned = false @volatile var driver: Option[RpcEndpointRef] = None // If this CoarseGrainedExecutorBackend is changed to support multiple threads, then this may need @@ -80,6 +79,8 @@ private[spark] class CoarseGrainedExecutorBackend( */ private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] + @volatile private var decommissioned = false + override def onStart(): Unit = { logInfo("Registering PWR handler.") SignalUtils.register("PWR", "Failed to register SIGPWR handler - " + @@ -166,11 +167,15 @@ private[spark] class CoarseGrainedExecutorBackend( exitExecutor(1, "Received LaunchTask command but executor was null") } else { if (decommissioned) { - logError("Asked to launch a task while decommissioned.") + val msg = "Asked to launch a task while decommissioned." + logError(msg) driver match { case Some(endpoint) => logInfo("Sending DecommissionExecutor to driver.") - endpoint.send(DecommissionExecutor(executorId)) + endpoint.send( + DecommissionExecutor( + executorId, + ExecutorDecommissionInfo(msg, isHostDecommissioned = false))) case _ => logError("No registered driver to send Decommission to.") } @@ -210,6 +215,10 @@ private[spark] class CoarseGrainedExecutorBackend( case UpdateDelegationTokens(tokenBytes) => logInfo(s"Received tokens of ${tokenBytes.length} bytes") SparkHadoopUtil.get.addDelegationTokens(tokenBytes, env.conf) + + case DecommissionSelf => + logInfo("Received decommission self") + decommissionSelf() } override def onDisconnected(remoteAddress: RpcAddress): Unit = { @@ -259,24 +268,73 @@ private[spark] class CoarseGrainedExecutorBackend( } private def decommissionSelf(): Boolean = { - logInfo("Decommissioning self w/sync") + val msg = "Decommissioning self w/sync" + logInfo(msg) try { decommissioned = true // Tell master we are are decommissioned so it stops trying to schedule us if (driver.nonEmpty) { - driver.get.askSync[Boolean](DecommissionExecutor(executorId)) + driver.get.askSync[Boolean](DecommissionExecutor( + executorId, ExecutorDecommissionInfo(msg, false))) } else { logError("No driver to message decommissioning.") } if (executor != null) { executor.decommission() } - logInfo("Done decommissioning self.") + // Shutdown the executor once all tasks are gone & any configured migrations completed. + // Detecting migrations completion doesn't need to be perfect and we want to minimize the + // overhead for executors that are not in decommissioning state as overall that will be + // more of the executors. For example, this will not catch a block which is already in + // the process of being put from a remote executor before migration starts. This trade-off + // is viewed as acceptable to minimize introduction of any new locking structures in critical + // code paths. + + val shutdownThread = new Thread("wait-for-blocks-to-migrate") { + override def run(): Unit = { + var lastTaskRunningTime = System.nanoTime() + val sleep_time = 1000 // 1s + + while (true) { + logInfo("Checking to see if we can shutdown.") + Thread.sleep(sleep_time) + if (executor == null || executor.numRunningTasks == 0) { + if (env.conf.get(STORAGE_DECOMMISSION_ENABLED)) { + logInfo("No running tasks, checking migrations") + val (migrationTime, allBlocksMigrated) = env.blockManager.lastMigrationInfo() + // We can only trust allBlocksMigrated boolean value if there were no tasks running + // since the start of computing it. + if (allBlocksMigrated && (migrationTime > lastTaskRunningTime)) { + logInfo("No running tasks, all blocks migrated, stopping.") + exitExecutor(0, "Finished decommissioning", notifyDriver = true) + } else { + logInfo("All blocks not yet migrated.") + } + } else { + logInfo("No running tasks, no block migration configured, stopping.") + exitExecutor(0, "Finished decommissioning", notifyDriver = true) + } + } else { + logInfo("Blocked from shutdown by running ${executor.numRunningtasks} tasks") + // If there is a running task it could store blocks, so make sure we wait for a + // migration loop to complete after the last task is done. + // Note: this is only advanced if there is a running task, if there + // is no running task but the blocks are not done migrating this does not + // move forward. + lastTaskRunningTime = System.nanoTime() + } + } + } + } + shutdownThread.setDaemon(true) + shutdownThread.start() + + logInfo("Will exit when finished decommissioning") // Return true since we are handling a signal true } catch { case e: Exception => - logError(s"Error ${e} during attempt to decommission self") + logError("Unexpected error while decommissioning self", e) false } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bc0f0c0a7b705..d22002917472a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -154,11 +154,6 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) - // Plugins need to load using a class loader that includes the executor's user classpath - private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { - PluginContainer(env, resources.asJava) - } - // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. private val maxDirectResultSize = Math.min( @@ -225,6 +220,13 @@ private[spark] class Executor( heartbeater.start() + // Plugins need to load using a class loader that includes the executor's user classpath. + // Plugins also needs to be initialized after the heartbeater started + // to avoid blocking to send heartbeat (see SPARK-32175). + private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { + PluginContainer(env, resources.asJava) + } + metricsPoller.start() private[executor] def numRunningTasks: Int = runningTasks.size() diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index b98c7436f9906..a295ef06a6376 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -275,7 +275,8 @@ private[spark] object ConfigEntry { val UNDEFINED = "" - private val knownConfigs = new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() + private[spark] val knownConfigs = + new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() def registerEntry(entry: ConfigEntry[_]): Unit = { val existing = knownConfigs.putIfAbsent(entry.key, entry) diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala index 646d8556538c2..90c59b079461c 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala @@ -29,7 +29,8 @@ private[spark] object Kryo { val KRYO_USER_REGISTRATORS = ConfigBuilder("spark.kryo.registrator") .version("0.5.0") .stringConf - .createOptional + .toSequence + .createWithDefault(Nil) val KRYO_CLASSES_TO_REGISTER = ConfigBuilder("spark.kryo.classesToRegister") .version("1.2.0") diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala b/core/src/main/scala/org/apache/spark/internal/config/UI.scala index 231eecf086bbe..fcbe2b9775841 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/UI.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/UI.scala @@ -17,6 +17,7 @@ package org.apache.spark.internal.config +import java.util.Locale import java.util.concurrent.TimeUnit import org.apache.spark.network.util.ByteUnit @@ -191,4 +192,15 @@ private[spark] object UI { .version("3.0.0") .stringConf .createOptional + + val MASTER_UI_DECOMMISSION_ALLOW_MODE = ConfigBuilder("spark.master.ui.decommission.allow.mode") + .doc("Specifies the behavior of the Master Web UI's /workers/kill endpoint. Possible choices" + + " are: `LOCAL` means allow this endpoint from IP's that are local to the machine running" + + " the Master, `DENY` means to completely disable this endpoint, `ALLOW` means to allow" + + " calling this endpoint from any IP.") + .internal() + .version("3.1.0") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .createWithDefault("LOCAL") } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index ca75a19af7bf6..200cde0a2d3ed 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -420,6 +420,29 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED = + ConfigBuilder("spark.storage.decommission.shuffleBlocks.enabled") + .doc("Whether to transfer shuffle blocks during block manager decommissioning. Requires " + + "a migratable shuffle resolver (like sort based shuffe)") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + + private[spark] val STORAGE_DECOMMISSION_SHUFFLE_MAX_THREADS = + ConfigBuilder("spark.storage.decommission.shuffleBlocks.maxThreads") + .doc("Maximum number of threads to use in migrating shuffle files.") + .version("3.1.0") + .intConf + .checkValue(_ > 0, "The maximum number of threads should be positive") + .createWithDefault(8) + + private[spark] val STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED = + ConfigBuilder("spark.storage.decommission.rddBlocks.enabled") + .doc("Whether to transfer RDD blocks during block manager decommissioning.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + private[spark] val STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK = ConfigBuilder("spark.storage.decommission.maxReplicationFailuresPerBlock") .internal() @@ -1843,6 +1866,17 @@ package object config { .timeConf(TimeUnit.MILLISECONDS) .createOptional + private[spark] val EXECUTOR_DECOMMISSION_KILL_INTERVAL = + ConfigBuilder("spark.executor.decommission.killInterval") + .doc("Duration after which a decommissioned executor will be killed forcefully." + + "This config is useful for cloud environments where we know in advance when " + + "an executor is going to go down after decommissioning signal i.e. around 2 mins " + + "in aws spot nodes, 1/2 hrs in spot block nodes etc. This config is currently " + + "used to decide what tasks running on decommission executors to speculate.") + .version("3.1.0") + .timeConf(TimeUnit.SECONDS) + .createOptional + private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir") .doc("Staging directory used while submitting applications.") .version("2.0.0") @@ -1874,4 +1908,11 @@ package object config { .version("3.1.0") .booleanConf .createWithDefault(false) + + private[spark] val EXECUTOR_ALLOW_SPARK_CONTEXT = + ConfigBuilder("spark.executor.allowSparkContext") + .doc("If set to true, SparkContext can be created in executors.") + .version("3.0.1") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index a619f10bbf064..6d174b5e0f81b 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -222,7 +222,9 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) if (path != null) { path.getFileSystem(getConf) } else { + // scalastyle:off FileSystemGet FileSystem.get(getConf) + // scalastyle:on FileSystemGet } } @@ -285,7 +287,9 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { // FileOutputFormat ignores the filesystem parameter + // scalastyle:off FileSystemGet val ignoredFs = FileSystem.get(getConf) + // scalastyle:on FileSystemGet getOutputFormat().checkOutputSpecs(ignoredFs, getConf) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 3de7377f99202..5d9cea068b097 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -168,7 +168,10 @@ private[spark] class NettyBlockTransferService( // Everything else is encoded using our binary protocol. val metadata = JavaUtils.bufferToArray(serializer.newInstance().serialize((level, classTag))) - val asStream = blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) + // We always transfer shuffle blocks as a stream for simplicity with the receiving code since + // they are always written to disk. Otherwise we check the block size. + val asStream = (blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) || + blockId.isShuffle) val callback = new RpcResponseCallback { override def onSuccess(response: ByteBuffer): Unit = { logTrace(s"Successfully uploaded block $blockId${if (asStream) " as stream" else ""}") diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index 9da6ffb1d2577..654afa01c7113 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -54,6 +54,20 @@ class ExecutorResourceRequests() extends Serializable { this } + /** + * Specify off heap memory. The value specified will be converted to MiB. + * This value only take effect when MEMORY_OFFHEAP_ENABLED is true. + * + * @param amount Amount of memory. In the same format as JVM memory strings (e.g. 512m, 2g). + * Default unit is MiB if not specified. + */ + def offHeapMemory(amount: String): this.type = { + val amountMiB = JavaUtils.byteStringAsMb(amount) + val req = new ExecutorResourceRequest(OFFHEAP_MEM, amountMiB) + _executorResources.put(OFFHEAP_MEM, req) + this + } + /** * Specify overhead memory. The value specified will be converted to MiB. * diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 22272a0f98a6c..482d9e94c6dd9 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -56,7 +56,7 @@ trait ResourceAllocator { def availableAddrs: Seq[String] = addressAvailabilityMap .flatMap { case (addr, available) => (0 until available).map(_ => addr) - }.toSeq + }.toSeq.sorted /** * Sequence of currently assigned resource addresses. @@ -68,7 +68,7 @@ trait ResourceAllocator { private[spark] def assignedAddrs: Seq[String] = addressAvailabilityMap .flatMap { case (addr, available) => (0 until slotsPerAddress - available).map(_ => addr) - }.toSeq + }.toSeq.sorted /** * Acquire a sequence of resource addresses (to a launched task), these addresses must be diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index f56ea69f6cec5..8a37670c31b9a 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -243,13 +243,15 @@ object ResourceProfile extends Logging { // task resources val CPUS = "cpus" // Executor resources + // Make sure add new executor resource in below allSupportedExecutorResources val CORES = "cores" val MEMORY = "memory" + val OFFHEAP_MEM = "offHeap" val OVERHEAD_MEM = "memoryOverhead" val PYSPARK_MEM = "pyspark.memory" // all supported spark executor resources (minus the custom resources like GPUs/FPGAs) - val allSupportedExecutorResources = Seq(CORES, MEMORY, OVERHEAD_MEM, PYSPARK_MEM) + val allSupportedExecutorResources = Seq(CORES, MEMORY, OVERHEAD_MEM, PYSPARK_MEM, OFFHEAP_MEM) val UNKNOWN_RESOURCE_PROFILE_ID = -1 val DEFAULT_RESOURCE_PROFILE_ID = 0 @@ -295,6 +297,10 @@ object ResourceProfile extends Logging { ereqs.memory(conf.get(EXECUTOR_MEMORY).toString) conf.get(EXECUTOR_MEMORY_OVERHEAD).map(mem => ereqs.memoryOverhead(mem.toString)) conf.get(PYSPARK_EXECUTOR_MEMORY).map(mem => ereqs.pysparkMemory(mem.toString)) + if (conf.get(MEMORY_OFFHEAP_ENABLED)) { + // Explicitly add suffix b as default unit of offHeapMemory is Mib + ereqs.offHeapMemory(conf.get(MEMORY_OFFHEAP_SIZE).toString + "b") + } val execReq = ResourceUtils.parseAllResourceRequests(conf, SPARK_EXECUTOR_PREFIX) execReq.foreach { req => val name = req.id.resourceName diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 41d6d146a86d7..4a9f551646fc7 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -122,7 +122,7 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte while (iter.hasNext) { val name = iter.next postMessage(name, message, (e) => { e match { - case e: RpcEnvStoppedException => logDebug (s"Message $message dropped. ${e.getMessage}") + case e: RpcEnvStoppedException => logDebug(s"Message $message dropped. ${e.getMessage}") case e: Throwable => logWarning(s"Message $message dropped. ${e.getMessage}") }} )} @@ -147,7 +147,14 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte /** Posts a one-way message. */ def postOneWayMessage(message: RequestMessage): Unit = { postMessage(message.receiver.name, OneWayMessage(message.senderAddress, message.content), - (e) => throw e) + (e) => e match { + // SPARK-31922: in local cluster mode, there's always a RpcEnvStoppedException when + // stop is called due to some asynchronous message handling. We catch the exception + // and log it at debug level to avoid verbose error message when user stop a local + // cluster in spark shell. + case re: RpcEnvStoppedException => logDebug(s"Message $message dropped. ${re.getMessage}") + case _ => throw e + }) } /** diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 3886cc5baa48e..fcb9fe422c0d4 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -213,7 +213,7 @@ private[netty] class NettyRpcEnv( def onFailure(e: Throwable): Unit = { if (!promise.tryFailure(e)) { e match { - case e : RpcEnvStoppedException => logDebug (s"Ignored failure: $e") + case e : RpcEnvStoppedException => logDebug(s"Ignored failure: $e") case _ => logWarning(s"Ignored failure: $e") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala index 2274e6898adf6..043c6b90384b4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala @@ -60,6 +60,6 @@ private[spark] object BarrierJobAllocationFailed { val ERROR_MESSAGE_BARRIER_REQUIRE_MORE_SLOTS_THAN_CURRENT_TOTAL_NUMBER = "[SPARK-24819]: Barrier execution mode does not allow run a barrier stage that requires " + "more slots than the total number of slots in the cluster currently. Please init a new " + - "cluster with more CPU cores or repartition the input RDD(s) to reduce the number of " + - "slots required to run this barrier stage." + "cluster with more resources(e.g. CPU, GPU) or repartition the input RDD(s) to reduce " + + "the number of slots required to run this barrier stage." } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index cb024d0852d06..7641948ed4b30 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -170,13 +170,34 @@ private[spark] class DAGScheduler( */ private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]] - // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with - // every task. When we detect a node failing, we note the current epoch number and failed - // executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask results. - // - // TODO: Garbage collect information about failure epochs when we know there are no more - // stray messages to detect. - private val failedEpoch = new HashMap[String, Long] + /** + * Tracks the latest epoch of a fully processed error related to the given executor. (We use + * the MapOutputTracker's epoch number, which is sent with every task.) + * + * When an executor fails, it can affect the results of many tasks, and we have to deal with + * all of them consistently. We don't simply ignore all future results from that executor, + * as the failures may have been transient; but we also don't want to "overreact" to follow- + * on errors we receive. Furthermore, we might receive notification of a task success, after + * we find out the executor has actually failed; we'll assume those successes are, in fact, + * simply delayed notifications and the results have been lost, if the tasks started in the + * same or an earlier epoch. In particular, we use this to control when we tell the + * BlockManagerMaster that the BlockManager has been lost. + */ + private val executorFailureEpoch = new HashMap[String, Long] + + /** + * Tracks the latest epoch of a fully processed error where shuffle files have been lost from + * the given executor. + * + * This is closely related to executorFailureEpoch. They only differ for the executor when + * there is an external shuffle service serving shuffle files and we haven't been notified that + * the entire worker has been lost. In that case, when an executor is lost, we do not update + * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only the executor + * fails, we do not unregister the shuffle data as it can still be served; but if there is + * a failure in the shuffle service (resulting in fetch failure), we unregister the shuffle + * data only once, even if we get many fetch failures. + */ + private val shuffleFileLostEpoch = new HashMap[String, Long] private [scheduler] val outputCommitCoordinator = env.outputCommitCoordinator @@ -311,6 +332,26 @@ private[spark] class DAGScheduler( eventProcessLoop.post(SpeculativeTaskSubmitted(task)) } + /** + * Called by the TaskSetManager when a taskset becomes unschedulable due to blacklisting and + * dynamic allocation is enabled. + */ + def unschedulableTaskSetAdded( + stageId: Int, + stageAttemptId: Int): Unit = { + eventProcessLoop.post(UnschedulableTaskSetAdded(stageId, stageAttemptId)) + } + + /** + * Called by the TaskSetManager when an unschedulable taskset becomes schedulable and dynamic + * allocation is enabled. + */ + def unschedulableTaskSetRemoved( + stageId: Int, + stageAttemptId: Int): Unit = { + eventProcessLoop.post(UnschedulableTaskSetRemoved(stageId, stageAttemptId)) + } + private[scheduler] def getCacheLocs(rdd: RDD[_]): IndexedSeq[Seq[TaskLocation]] = cacheLocs.synchronized { // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times @@ -439,10 +480,12 @@ private[spark] class DAGScheduler( * submission. */ private def checkBarrierStageWithNumSlots(rdd: RDD[_], rp: ResourceProfile): Unit = { - val numPartitions = rdd.getNumPartitions - val maxNumConcurrentTasks = sc.maxNumConcurrentTasks(rp) - if (rdd.isBarrier() && numPartitions > maxNumConcurrentTasks) { - throw new BarrierJobSlotsNumberCheckFailed(numPartitions, maxNumConcurrentTasks) + if (rdd.isBarrier()) { + val numPartitions = rdd.getNumPartitions + val maxNumConcurrentTasks = sc.maxNumConcurrentTasks(rp) + if (numPartitions > maxNumConcurrentTasks) { + throw new BarrierJobSlotsNumberCheckFailed(numPartitions, maxNumConcurrentTasks) + } } } @@ -1014,6 +1057,18 @@ private[spark] class DAGScheduler( listenerBus.post(SparkListenerSpeculativeTaskSubmitted(task.stageId, task.stageAttemptId)) } + private[scheduler] def handleUnschedulableTaskSetAdded( + stageId: Int, + stageAttemptId: Int): Unit = { + listenerBus.post(SparkListenerUnschedulableTaskSetAdded(stageId, stageAttemptId)) + } + + private[scheduler] def handleUnschedulableTaskSetRemoved( + stageId: Int, + stageAttemptId: Int): Unit = { + listenerBus.post(SparkListenerUnschedulableTaskSetRemoved(stageId, stageAttemptId)) + } + private[scheduler] def handleTaskSetFailed( taskSet: TaskSet, reason: String, @@ -1566,7 +1621,8 @@ private[spark] class DAGScheduler( val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) - if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { + if (executorFailureEpoch.contains(execId) && + smt.epoch <= executorFailureEpoch(execId)) { logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") } else { // The epoch of the task is acceptable (i.e., the task was launched after the most @@ -1767,10 +1823,19 @@ private[spark] class DAGScheduler( // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { - val hostToUnregisterOutputs = if (env.blockManager.externalShuffleServiceEnabled && - unRegisterOutputOnHostOnFetchFailure) { - // We had a fetch failure with the external shuffle service, so we - // assume all shuffle data on the node is bad. + val externalShuffleServiceEnabled = env.blockManager.externalShuffleServiceEnabled + val isHostDecommissioned = taskScheduler + .getExecutorDecommissionInfo(bmAddress.executorId) + .exists(_.isHostDecommissioned) + + // Shuffle output of all executors on host `bmAddress.host` may be lost if: + // - External shuffle service is enabled, so we assume that all shuffle data on node is + // bad. + // - Host is decommissioned, thus all executors on that host will die. + val shuffleOutputOfEntireHostLost = externalShuffleServiceEnabled || + isHostDecommissioned + val hostToUnregisterOutputs = if (shuffleOutputOfEntireHostLost + && unRegisterOutputOnHostOnFetchFailure) { Some(bmAddress.host) } else { // Unregister shuffle data just for one executor (we don't have any @@ -1912,12 +1977,8 @@ private[spark] class DAGScheduler( * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * * We will also assume that we've lost all shuffle blocks associated with the executor if the - * executor serves its own blocks (i.e., we're not using external shuffle), the entire executor - * process is lost (likely including the shuffle service), or a FetchFailed occurred, in which - * case we presume all shuffle data related to this executor to be lost. - * - * Optionally the epoch during which the failure was caught can be passed to avoid allowing - * stray fetch failures from possibly retriggering the detection of a node as lost. + * executor serves its own blocks (i.e., we're not using an external shuffle service), or the + * entire Standalone worker is lost. */ private[scheduler] def handleExecutorLost( execId: String, @@ -1933,29 +1994,44 @@ private[spark] class DAGScheduler( maybeEpoch = None) } + /** + * Handles removing an executor from the BlockManagerMaster as well as unregistering shuffle + * outputs for the executor or optionally its host. + * + * @param execId executor to be removed + * @param fileLost If true, indicates that we assume we've lost all shuffle blocks associated + * with the executor; this happens if the executor serves its own blocks (i.e., we're not + * using an external shuffle service), the entire Standalone worker is lost, or a FetchFailed + * occurred (in which case we presume all shuffle data related to this executor to be lost). + * @param hostToUnregisterOutputs (optional) executor host if we're unregistering all the + * outputs on the host + * @param maybeEpoch (optional) the epoch during which the failure was caught (this prevents + * reprocessing for follow-on fetch failures) + */ private def removeExecutorAndUnregisterOutputs( execId: String, fileLost: Boolean, hostToUnregisterOutputs: Option[String], maybeEpoch: Option[Long] = None): Unit = { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) - if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { - failedEpoch(execId) = currentEpoch - logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch)) + logDebug(s"Considering removal of executor $execId; " + + s"fileLost: $fileLost, currentEpoch: $currentEpoch") + if (!executorFailureEpoch.contains(execId) || executorFailureEpoch(execId) < currentEpoch) { + executorFailureEpoch(execId) = currentEpoch + logInfo(s"Executor lost: $execId (epoch $currentEpoch)") blockManagerMaster.removeExecutor(execId) - if (fileLost) { - hostToUnregisterOutputs match { - case Some(host) => - logInfo("Shuffle files lost for host: %s (epoch %d)".format(host, currentEpoch)) - mapOutputTracker.removeOutputsOnHost(host) - case None => - logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch)) - mapOutputTracker.removeOutputsOnExecutor(execId) - } - clearCacheLocs() - - } else { - logDebug("Additional executor lost message for %s (epoch %d)".format(execId, currentEpoch)) + clearCacheLocs() + } + if (fileLost && + (!shuffleFileLostEpoch.contains(execId) || shuffleFileLostEpoch(execId) < currentEpoch)) { + shuffleFileLostEpoch(execId) = currentEpoch + hostToUnregisterOutputs match { + case Some(host) => + logInfo(s"Shuffle files lost for host: $host (epoch $currentEpoch)") + mapOutputTracker.removeOutputsOnHost(host) + case None => + logInfo(s"Shuffle files lost for executor: $execId (epoch $currentEpoch)") + mapOutputTracker.removeOutputsOnExecutor(execId) } } } @@ -1981,11 +2057,12 @@ private[spark] class DAGScheduler( } private[scheduler] def handleExecutorAdded(execId: String, host: String): Unit = { - // remove from failedEpoch(execId) ? - if (failedEpoch.contains(execId)) { + // remove from executorFailureEpoch(execId) ? + if (executorFailureEpoch.contains(execId)) { logInfo("Host added was in lost list earlier: " + host) - failedEpoch -= execId + executorFailureEpoch -= execId } + shuffleFileLostEpoch -= execId } private[scheduler] def handleStageCancellation(stageId: Int, reason: Option[String]): Unit = { @@ -2273,7 +2350,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case ExecutorLost(execId, reason) => val workerLost = reason match { - case ExecutorProcessLost(_, true) => true + case ExecutorProcessLost(_, true, _) => true case _ => false } dagScheduler.handleExecutorLost(execId, workerLost) @@ -2287,6 +2364,12 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case SpeculativeTaskSubmitted(task) => dagScheduler.handleSpeculativeTaskSubmitted(task) + case UnschedulableTaskSetAdded(stageId, stageAttemptId) => + dagScheduler.handleUnschedulableTaskSetAdded(stageId, stageAttemptId) + + case UnschedulableTaskSetRemoved(stageId, stageAttemptId) => + dagScheduler.handleUnschedulableTaskSetRemoved(stageId, stageAttemptId) + case GettingResultEvent(taskInfo) => dagScheduler.handleGetTaskResult(taskInfo) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 78d458338e8fb..d226fe88614d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -97,3 +97,11 @@ private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent private[scheduler] case class SpeculativeTaskSubmitted(task: Task[_]) extends DAGSchedulerEvent +private[scheduler] +case class UnschedulableTaskSetAdded(stageId: Int, stageAttemptId: Int) + extends DAGSchedulerEvent + +private[scheduler] +case class UnschedulableTaskSetRemoved(stageId: Int, stageAttemptId: Int) + extends DAGSchedulerEvent + diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala new file mode 100644 index 0000000000000..a82b5d38afe9f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +/** + * Provides more detail when an executor is being decommissioned. + * @param message Human readable reason for why the decommissioning is happening. + * @param isHostDecommissioned Whether the host (aka the `node` or `worker` in other places) is + * being decommissioned too. Used to infer if the shuffle data might + * be lost even if the external shuffle service is enabled. + */ +private[spark] +case class ExecutorDecommissionInfo(message: String, isHostDecommissioned: Boolean) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index 4141ed799a4e0..671dedaa5a6e8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -54,9 +54,14 @@ private [spark] object LossReasonPending extends ExecutorLossReason("Pending los /** * @param _message human readable loss reason * @param workerLost whether the worker is confirmed lost too (i.e. including shuffle service) + * @param causedByApp whether the loss of the executor is the fault of the running app. + * (assumed true by default unless known explicitly otherwise) */ private[spark] -case class ExecutorProcessLost(_message: String = "Worker lost", workerLost: Boolean = false) +case class ExecutorProcessLost( + _message: String = "Executor Process Lost", + workerLost: Boolean = false, + causedByApp: Boolean = true) extends ExecutorLossReason(_message) /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index fd04db8c09d76..508c6cebd9fe3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -36,4 +36,5 @@ private[spark] class ExecutorResourceInfo( override protected def resourceName = this.name override protected def resourceAddresses = this.addresses override protected def slotsPerAddress: Int = numParts + def totalAddressAmount: Int = resourceAddresses.length * slotsPerAddress } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 7f8893ff3b9d8..cfc2e141290c4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -30,12 +30,15 @@ import org.apache.spark.util.Utils /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the - * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. + * task has shuffle files stored on as well as the sizes of outputs for each reducer, for passing + * on to the reduce tasks. */ private[spark] sealed trait MapStatus { - /** Location where this task was run. */ + /** Location where this task output is. */ def location: BlockManagerId + def updateLocation(newLoc: BlockManagerId): Unit + /** * Estimated size for the reduce block, in bytes. * @@ -126,6 +129,10 @@ private[spark] class CompressedMapStatus( override def location: BlockManagerId = loc + override def updateLocation(newLoc: BlockManagerId): Unit = { + loc = newLoc + } + override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) } @@ -178,6 +185,10 @@ private[spark] class HighlyCompressedMapStatus private ( override def location: BlockManagerId = loc + override def updateLocation(newLoc: BlockManagerId): Unit = { + loc = newLoc + } + override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { @@ -194,7 +205,7 @@ private[spark] class HighlyCompressedMapStatus private ( override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) - emptyBlocks.writeExternal(out) + emptyBlocks.serialize(out) out.writeLong(avgSize) out.writeInt(hugeBlockSizes.size) hugeBlockSizes.foreach { kv => @@ -206,8 +217,9 @@ private[spark] class HighlyCompressedMapStatus private ( override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) + numNonEmptyBlocks = -1 // SPARK-32436 Scala 2.13 doesn't initialize this during deserialization emptyBlocks = new RoaringBitmap() - emptyBlocks.readExternal(in) + emptyBlocks.deserialize(in) avgSize = in.readLong() val count = in.readInt() val hugeBlockSizesImpl = mutable.Map.empty[Int, Byte] diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index a5bba645be14c..a566d0a04387c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -83,7 +83,7 @@ private[spark] trait SchedulerBackend { /** * Get the max number of tasks that can be concurrent launched based on the ResourceProfile - * being used. + * could be used, even if some of them are being used at the moment. * Note that please don't cache the value returned by this method, because the number can change * due to add/remove executors. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 62d54f3b74a47..8119215b8b74f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -157,6 +157,16 @@ case class SparkListenerNodeBlacklisted( case class SparkListenerNodeUnblacklisted(time: Long, hostId: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerUnschedulableTaskSetAdded( + stageId: Int, + stageAttemptId: Int) extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerUnschedulableTaskSetRemoved( + stageId: Int, + stageAttemptId: Int) extends SparkListenerEvent + @DeveloperApi case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends SparkListenerEvent @@ -339,6 +349,20 @@ private[spark] trait SparkListenerInterface { */ def onNodeUnblacklisted(nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit + /** + * Called when a taskset becomes unschedulable due to blacklisting and dynamic allocation + * is enabled. + */ + def onUnschedulableTaskSetAdded( + unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded): Unit + + /** + * Called when an unschedulable taskset becomes schedulable and dynamic allocation + * is enabled. + */ + def onUnschedulableTaskSetRemoved( + unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved): Unit + /** * Called when the driver receives a block update info. */ @@ -425,6 +449,12 @@ abstract class SparkListener extends SparkListenerInterface { override def onNodeUnblacklisted( nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit = { } + override def onUnschedulableTaskSetAdded( + unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded): Unit = { } + + override def onUnschedulableTaskSetRemoved( + unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved): Unit = { } + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { } override def onSpeculativeTaskSubmitted( diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 3d316c948db7e..13e65f4291fd0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -79,6 +79,10 @@ private[spark] trait SparkListenerBus listener.onBlockUpdated(blockUpdated) case speculativeTaskSubmitted: SparkListenerSpeculativeTaskSubmitted => listener.onSpeculativeTaskSubmitted(speculativeTaskSubmitted) + case unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded => + listener.onUnschedulableTaskSetAdded(unschedulableTaskSetAdded) + case unschedulableTaskSetRemoved: SparkListenerUnschedulableTaskSetRemoved => + listener.onUnschedulableTaskSetRemoved(unschedulableTaskSetRemoved) case resourceProfileAdded: SparkListenerResourceProfileAdded => listener.onResourceProfileAdded(resourceProfileAdded) case _ => listener.onOtherEvent(event) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 08f9f3c256e69..1101d0616d2bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -101,7 +101,12 @@ private[spark] trait TaskScheduler { /** * Process a decommissioning executor. */ - def executorDecommission(executorId: String): Unit + def executorDecommission(executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit + + /** + * If an executor is decommissioned, return its corresponding decommission info + */ + def getExecutorDecommissionInfo(executorId: String): Option[ExecutorDecommissionInfo] /** * Process a lost executor diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 12bd93286d736..a0c507e7f893b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -136,6 +136,8 @@ private[spark] class TaskSchedulerImpl( // IDs of the tasks running on each executor private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]] + private val executorsPendingDecommission = new HashMap[String, ExecutorDecommissionInfo] + def runningTasksByExecutors: Map[String, Int] = synchronized { executorIdToRunningTaskIds.toMap.mapValues(_.size).toMap } @@ -466,51 +468,6 @@ private[spark] class TaskSchedulerImpl( Some(localTaskReqAssign.toMap) } - // Use the resource that the resourceProfile has as the limiting resource to calculate the - // total number of slots available based on the current offers. - private def calculateAvailableSlots( - resourceProfileIds: Array[Int], - availableCpus: Array[Int], - availableResources: Array[Map[String, Buffer[String]]], - taskSet: TaskSetManager): Int = { - val resourceProfile = sc.resourceProfileManager.resourceProfileFromId( - taskSet.taskSet.resourceProfileId) - val offersForResourceProfile = resourceProfileIds.zipWithIndex.filter { case (id, _) => - (id == resourceProfile.id) - } - val coresKnown = resourceProfile.isCoresLimitKnown - var limitingResource = resourceProfile.limitingResource(conf) - val taskCpus = ResourceProfile.getTaskCpusOrDefaultForProfile(resourceProfile, conf) - - offersForResourceProfile.map { case (o, index) => - val numTasksPerExecCores = availableCpus(index) / taskCpus - // if limiting resource is empty then we have no other resources, so it has to be CPU - if (limitingResource == ResourceProfile.CPUS || limitingResource.isEmpty) { - numTasksPerExecCores - } else { - val taskLimit = resourceProfile.taskResources.get(limitingResource).map(_.amount) - .getOrElse { - val errorMsg = "limitingResource returns from ResourceProfile " + - s"$resourceProfile doesn't actually contain that task resource!" - taskSet.abort(errorMsg) - throw new SparkException(errorMsg) - } - // available addresses already takes into account if there are fractional - // task resource requests - val availAddrs = availableResources(index).get(limitingResource).map(_.size).getOrElse(0) - val resourceLimit = (availAddrs / taskLimit).toInt - if (!coresKnown) { - // when executor cores config isn't set, we can't calculate the real limiting resource - // and number of tasks per executor ahead of time, so calculate it now based on what - // is available. - if (numTasksPerExecCores <= resourceLimit) numTasksPerExecCores else resourceLimit - } else { - resourceLimit - } - } - }.sum - } - private def minTaskLocality( l1: Option[TaskLocality], l2: Option[TaskLocality]) : Option[TaskLocality] = { @@ -589,9 +546,14 @@ private[spark] class TaskSchedulerImpl( // we only need to calculate available slots if using barrier scheduling, otherwise the // value is -1 val numBarrierSlotsAvailable = if (taskSet.isBarrier) { - val slots = calculateAvailableSlots(resourceProfileIds, availableCpus, availableResources, - taskSet) - slots + val rpId = taskSet.taskSet.resourceProfileId + val availableResourcesAmount = availableResources.map { resourceMap => + // available addresses already takes into account if there are fractional + // task resource requests + resourceMap.map { case (name, addresses) => (name, addresses.length) } + } + calculateAvailableSlots(this, conf, rpId, resourceProfileIds, availableCpus, + availableResourcesAmount) } else { -1 } @@ -637,10 +599,9 @@ private[spark] class TaskSchedulerImpl( if (!launchedAnyTask) { taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors).foreach { taskIndex => // If the taskSet is unschedulable we try to find an existing idle blacklisted - // executor. If we cannot find one, we abort immediately. Else we kill the idle - // executor and kick off an abortTimer which if it doesn't schedule a task within the - // the timeout will abort the taskSet if we were unable to schedule any task from the - // taskSet. + // executor and kill the idle executor and kick off an abortTimer which if it doesn't + // schedule a task within the the timeout will abort the taskSet if we were unable to + // schedule any task from the taskSet. // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per // task basis. // Note 2: The taskSet can still be aborted when there are more than one idle @@ -648,22 +609,34 @@ private[spark] class TaskSchedulerImpl( // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort // timer to expire and abort the taskSet. + // + // If there are no idle executors and dynamic allocation is enabled, then we would + // notify ExecutorAllocationManager to allocate more executors to schedule the + // unschedulable tasks else we will abort immediately. executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match { case Some ((executorId, _)) => if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId)) - - val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 - unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout - logInfo(s"Waiting for $timeout ms for completely " - + s"blacklisted task to be schedulable again before aborting $taskSet.") - abortTimer.schedule( - createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) + updateUnschedulableTaskSetTimeoutAndStartAbortTimer(taskSet, taskIndex) + } + case None => + // Notify ExecutorAllocationManager about the unschedulable task set, + // in order to provision more executors to make them schedulable + if (Utils.isDynamicAllocationEnabled(conf)) { + if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { + logInfo("Notifying ExecutorAllocationManager to allocate more executors to" + + " schedule the unschedulable task before aborting" + + " stage ${taskSet.stageId}.") + dagScheduler.unschedulableTaskSetAdded(taskSet.taskSet.stageId, + taskSet.taskSet.stageAttemptId) + updateUnschedulableTaskSetTimeoutAndStartAbortTimer(taskSet, taskIndex) + } + } else { + // Abort Immediately + logInfo("Cannot schedule any task because of complete blacklisting. No idle" + + s" executors can be found to kill. Aborting stage ${taskSet.stageId}.") + taskSet.abortSinceCompletelyBlacklisted(taskIndex) } - case None => // Abort Immediately - logInfo("Cannot schedule any task because of complete blacklisting. No idle" + - s" executors can be found to kill. Aborting $taskSet." ) - taskSet.abortSinceCompletelyBlacklisted(taskIndex) } } } else { @@ -676,6 +649,10 @@ private[spark] class TaskSchedulerImpl( if (unschedulableTaskSetToExpiryTime.nonEmpty) { logInfo("Clearing the expiry times for all unschedulable taskSets as a task was " + "recently scheduled.") + // Notify ExecutorAllocationManager as well as other subscribers that a task now + // recently becomes schedulable + dagScheduler.unschedulableTaskSetRemoved(taskSet.taskSet.stageId, + taskSet.taskSet.stageAttemptId) unschedulableTaskSetToExpiryTime.clear() } } @@ -722,6 +699,17 @@ private[spark] class TaskSchedulerImpl( return tasks.map(_.toSeq) } + private def updateUnschedulableTaskSetTimeoutAndStartAbortTimer( + taskSet: TaskSetManager, + taskIndex: Int): Unit = { + val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 + unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout + logInfo(s"Waiting for $timeout ms for completely " + + s"blacklisted task to be schedulable again before aborting stage ${taskSet.stageId}.") + abortTimer.schedule( + createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) + } + private def createUnschedulableTaskSetAbortTimer( taskSet: TaskSetManager, taskIndex: Int): TimerTask = { @@ -730,7 +718,7 @@ private[spark] class TaskSchedulerImpl( if (unschedulableTaskSetToExpiryTime.contains(taskSet) && unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()) { logInfo("Cannot schedule any task because of complete blacklisting. " + - s"Wait time for scheduling expired. Aborting $taskSet.") + s"Wait time for scheduling expired. Aborting stage ${taskSet.stageId}.") taskSet.abortSinceCompletelyBlacklisted(taskIndex) } else { this.cancel() @@ -912,13 +900,45 @@ private[spark] class TaskSchedulerImpl( } } - override def executorDecommission(executorId: String): Unit = { + override def executorDecommission( + executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { + synchronized { + // Don't bother noting decommissioning for executors that we don't know about + if (executorIdToHost.contains(executorId)) { + // The scheduler can get multiple decommission updates from multiple sources, + // and some of those can have isHostDecommissioned false. We merge them such that + // if we heard isHostDecommissioned ever true, then we keep that one since it is + // most likely coming from the cluster manager and thus authoritative + val oldDecomInfo = executorsPendingDecommission.get(executorId) + if (oldDecomInfo.isEmpty || !oldDecomInfo.get.isHostDecommissioned) { + executorsPendingDecommission(executorId) = decommissionInfo + } + } + } rootPool.executorDecommission(executorId) backend.reviveOffers() } - override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = { + override def getExecutorDecommissionInfo(executorId: String) + : Option[ExecutorDecommissionInfo] = synchronized { + executorsPendingDecommission.get(executorId) + } + + override def executorLost(executorId: String, givenReason: ExecutorLossReason): Unit = { var failedExecutor: Option[String] = None + val reason = givenReason match { + // Handle executor process loss due to decommissioning + case ExecutorProcessLost(message, origWorkerLost, origCausedByApp) => + val executorDecommissionInfo = getExecutorDecommissionInfo(executorId) + ExecutorProcessLost( + message, + // Also mark the worker lost if we know that the host was decommissioned + origWorkerLost || executorDecommissionInfo.exists(_.isHostDecommissioned), + // Executor loss is certainly not caused by app if we knew that this executor is being + // decommissioned + causedByApp = executorDecommissionInfo.isEmpty && origCausedByApp) + case e => e + } synchronized { if (executorIdToRunningTaskIds.contains(executorId)) { @@ -1007,6 +1027,8 @@ private[spark] class TaskSchedulerImpl( } } + executorsPendingDecommission -= executorId + if (reason != LossReasonPending) { executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) @@ -1104,6 +1126,63 @@ private[spark] object TaskSchedulerImpl { val SCHEDULER_MODE_PROPERTY = SCHEDULER_MODE.key + /** + * Calculate the max available task slots given the `availableCpus` and `availableResources` + * from a collection of ResourceProfiles. And only those ResourceProfiles who has the + * same id with the `rpId` can be used to calculate the task slots. + * + * @param scheduler the TaskSchedulerImpl instance + * @param conf SparkConf used to calculate the limiting resource and get the cpu amount per task + * @param rpId the target ResourceProfile id. Only those ResourceProfiles who has the same id + * with it can be used to calculate the task slots. + * @param availableRPIds an Array of ids of the available ResourceProfiles from the executors. + * @param availableCpus an Array of the amount of available cpus from the executors. + * @param availableResources an Array of the resources map from the executors. In the resource + * map, it maps from the resource name to its amount. + * @return the number of max task slots + */ + def calculateAvailableSlots( + scheduler: TaskSchedulerImpl, + conf: SparkConf, + rpId: Int, + availableRPIds: Array[Int], + availableCpus: Array[Int], + availableResources: Array[Map[String, Int]]): Int = { + val resourceProfile = scheduler.sc.resourceProfileManager.resourceProfileFromId(rpId) + val coresKnown = resourceProfile.isCoresLimitKnown + val (limitingResource, limitedByCpu) = { + val limiting = resourceProfile.limitingResource(conf) + // if limiting resource is empty then we have no other resources, so it has to be CPU + if (limiting == ResourceProfile.CPUS || limiting.isEmpty) { + (ResourceProfile.CPUS, true) + } else { + (limiting, false) + } + } + val cpusPerTask = ResourceProfile.getTaskCpusOrDefaultForProfile(resourceProfile, conf) + val taskLimit = resourceProfile.taskResources.get(limitingResource).map(_.amount).get + + availableCpus.zip(availableResources).zip(availableRPIds) + .filter { case (_, id) => id == rpId } + .map { case ((cpu, resources), _) => + val numTasksPerExecCores = cpu / cpusPerTask + if (limitedByCpu) { + numTasksPerExecCores + } else { + val availAddrs = resources.getOrElse(limitingResource, 0) + val resourceLimit = (availAddrs / taskLimit).toInt + // when executor cores config isn't set, we can't calculate the real limiting resource + // and number of tasks per executor ahead of time, so calculate it now based on what + // is available. + if (!coresKnown && numTasksPerExecCores <= resourceLimit) { + numTasksPerExecCores + } else { + resourceLimit + } + } + }.sum + } + /** * Used to balance containers across hosts. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a302f680a272e..d69f358cd19de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.{ConcurrentLinkedQueue, TimeUnit} import scala.collection.immutable.Map import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -102,6 +102,8 @@ private[spark] class TaskSetManager( } numTasks <= slots } + val executorDecommissionKillInterval = conf.get(EXECUTOR_DECOMMISSION_KILL_INTERVAL).map( + TimeUnit.SECONDS.toMillis) // For each task, tracks whether a copy of the task has succeeded. A task will also be // marked as "succeeded" if it failed with a fetch failure, in which case it should not @@ -165,6 +167,7 @@ private[spark] class TaskSetManager( // Task index, start and finish time for each task attempt (indexed by task ID) private[scheduler] val taskInfos = new HashMap[Long, TaskInfo] + private[scheduler] val tidToExecutorKillTimeMapping = new HashMap[Long, Long] // Use a MedianHeap to record durations of successful tasks so we know when to launch // speculative tasks. This is only used when speculation is enabled, to avoid the overhead @@ -933,6 +936,7 @@ private[spark] class TaskSetManager( /** If the given task ID is in the set of running tasks, removes it. */ def removeRunningTask(tid: Long): Unit = { + tidToExecutorKillTimeMapping.remove(tid) if (runningTasksSet.remove(tid) && parent != null) { parent.decreaseRunningTasks(1) } @@ -981,6 +985,7 @@ private[spark] class TaskSetManager( val exitCausedByApp: Boolean = reason match { case exited: ExecutorExited => exited.exitCausedByApp case ExecutorKilled => false + case ExecutorProcessLost(_, _, false) => false case _ => true } handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure(info.executorId, exitCausedByApp, @@ -1042,7 +1047,19 @@ private[spark] class TaskSetManager( // bound based on that. logDebug("Task length threshold for speculation: " + threshold) for (tid <- runningTasksSet) { - foundTasks |= checkAndSubmitSpeculatableTask(tid, time, threshold) + var speculated = checkAndSubmitSpeculatableTask(tid, time, threshold) + if (!speculated && tidToExecutorKillTimeMapping.contains(tid)) { + // Check whether this task will finish before the exectorKillTime assuming + // it will take medianDuration overall. If this task cannot finish within + // executorKillInterval, then this task is a candidate for speculation + val taskEndTimeBasedOnMedianDuration = taskInfos(tid).launchTime + medianDuration + val canExceedDeadline = tidToExecutorKillTimeMapping(tid) < + taskEndTimeBasedOnMedianDuration + if (canExceedDeadline) { + speculated = checkAndSubmitSpeculatableTask(tid, time, 0) + } + } + foundTasks |= speculated } } else if (speculationTaskDurationThresOpt.isDefined && speculationTasksLessEqToSlots) { val time = clock.getTimeMillis() @@ -1100,8 +1117,12 @@ private[spark] class TaskSetManager( def executorDecommission(execId: String): Unit = { recomputeLocality() - // Future consideration: if an executor is decommissioned it may make sense to add the current - // tasks to the spec exec queue. + executorDecommissionKillInterval.foreach { interval => + val executorKillTime = clock.getTimeMillis() + interval + runningTasksSet.filter(taskInfos(_).executorId == execId).foreach { tid => + tidToExecutorKillTimeMapping(tid) = executorKillTime + } + } } def recomputeLocality(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index bb929c27b6a65..7242ab7786061 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.scheduler.ExecutorDecommissionInfo import org.apache.spark.scheduler.ExecutorLossReason import org.apache.spark.util.SerializableBuffer @@ -94,7 +95,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: ExecutorLossReason) extends CoarseGrainedClusterMessage - case class DecommissionExecutor(executorId: String) extends CoarseGrainedClusterMessage + case class DecommissionExecutor(executorId: String, decommissionInfo: ExecutorDecommissionInfo) + extends CoarseGrainedClusterMessage case class RemoveWorker(workerId: String, host: String, message: String) extends CoarseGrainedClusterMessage @@ -134,4 +136,7 @@ private[spark] object CoarseGrainedClusterMessages { // The message to check if `CoarseGrainedSchedulerBackend` thinks the executor is alive or not. case class IsExecutorAlive(executorId: String) extends CoarseGrainedClusterMessage + + // Used to ask an executor to decommission itself. (Can be an internal message) + case object DecommissionSelf extends CoarseGrainedClusterMessage } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6b9b4d6fe57e0..200f2d87a8a7a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -191,9 +191,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) - case DecommissionExecutor(executorId) => - logError(s"Received decommission executor message ${executorId}.") - decommissionExecutor(executorId) + case DecommissionExecutor(executorId, decommissionInfo) => + logError(s"Received decommission executor message ${executorId}: $decommissionInfo") + decommissionExecutor(executorId, decommissionInfo) case RemoveWorker(workerId, host, message) => removeWorker(workerId, host, message) @@ -272,9 +272,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp removeWorker(workerId, host, message) context.reply(true) - case DecommissionExecutor(executorId) => - logError(s"Received decommission executor message ${executorId}.") - decommissionExecutor(executorId) + case DecommissionExecutor(executorId, decommissionInfo) => + logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.") + decommissionExecutor(executorId, decommissionInfo) context.reply(true) case RetrieveSparkAppConfig(resourceProfileId) => @@ -422,7 +422,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Mark a given executor as decommissioned and stop making resource offers for it. */ - private def decommissionExecutor(executorId: String): Boolean = { + private def decommissionExecutor( + executorId: String, decommissionInfo: ExecutorDecommissionInfo): Boolean = { val shouldDisable = CoarseGrainedSchedulerBackend.this.synchronized { // Only bother decommissioning executors which are alive. if (isExecutorActive(executorId)) { @@ -436,11 +437,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp if (shouldDisable) { logInfo(s"Starting decommissioning executor $executorId.") try { - scheduler.executorDecommission(executorId) + scheduler.executorDecommission(executorId, decommissionInfo) } catch { case e: Exception => logError(s"Unexpected error during decommissioning ${e.toString}", e) } + // Send decommission message to the executor, this may be a duplicate since the executor + // could have been the one to notify us. But it's also possible the notification came from + // elsewhere and the executor does not yet know. + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorInfo.executorEndpoint.send(DecommissionSelf) + case None => + // Ignoring the executor since it is not registered. + logWarning(s"Attempted to decommission unknown executor $executorId.") + } logInfo(s"Finished decommissioning executor $executorId.") if (conf.get(STORAGE_DECOMMISSION_ENABLED)) { @@ -562,7 +573,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } - override def reviveOffers(): Unit = { + override def reviveOffers(): Unit = Utils.tryLogNonFatalError { driverEndpoint.send(ReviveOffers) } @@ -590,10 +601,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Called by subclasses when notified of a decommissioning executor. */ - private[spark] def decommissionExecutor(executorId: String): Unit = { + private[spark] def decommissionExecutor( + executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { if (driverEndpoint != null) { logInfo("Propagating executor decommission to driver.") - driverEndpoint.send(DecommissionExecutor(executorId)) + driverEndpoint.send(DecommissionExecutor(executorId, decommissionInfo)) } } @@ -630,10 +642,28 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } + /** + * Get the max number of tasks that can be concurrent launched based on the ResourceProfile + * could be used, even if some of them are being used at the moment. + * Note that please don't cache the value returned by this method, because the number can change + * due to add/remove executors. + * + * @param rp ResourceProfile which to use to calculate max concurrent tasks. + * @return The max number of tasks that can be concurrent launched currently. + */ override def maxNumConcurrentTasks(rp: ResourceProfile): Int = synchronized { - val cpusPerTask = ResourceProfile.getTaskCpusOrDefaultForProfile(rp, conf) - val executorsWithResourceProfile = executorDataMap.values.filter(_.resourceProfileId == rp.id) - executorsWithResourceProfile.map(_.totalCores / cpusPerTask).sum + val (rpIds, cpus, resources) = { + executorDataMap + .filter { case (id, _) => isExecutorActive(id) } + .values.toArray.map { executor => + ( + executor.resourceProfileId, + executor.totalCores, + executor.resourcesInfo.map { case (name, rInfo) => (name, rInfo.totalAddressAmount) } + ) + }.unzip3 + } + TaskSchedulerImpl.calculateAvailableSlots(scheduler, conf, rp.id, rpIds, cpus, resources) } // this function is for testing only diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index ec1299a924b5c..d921af602b254 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -44,7 +44,7 @@ private[spark] class StandaloneSchedulerBackend( with StandaloneAppClientListener with Logging { - private var client: StandaloneAppClient = null + private[spark] var client: StandaloneAppClient = null private val stopping = new AtomicBoolean(false) private val launcherBackend = new LauncherBackend() { override protected def conf: SparkConf = sc.conf @@ -174,10 +174,10 @@ private[spark] class StandaloneSchedulerBackend( removeExecutor(fullId.split("/")(1), reason) } - override def executorDecommissioned(fullId: String, message: String) { + override def executorDecommissioned(fullId: String, decommissionInfo: ExecutorDecommissionInfo) { logInfo("Asked to decommission executor") - decommissionExecutor(fullId.split("/")(1)) - logInfo("Executor %s decommissioned: %s".format(fullId, message)) + decommissionExecutor(fullId.split("/")(1), decommissionInfo) + logInfo("Executor %s decommissioned: %s".format(fullId, decommissionInfo)) } override def workerRemoved(workerId: String, host: String, message: String): Unit = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index af2c82e771970..0d0dad6d77ac1 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle import java.io._ +import java.nio.ByteBuffer import java.nio.channels.Channels import java.nio.file.Files @@ -25,8 +26,10 @@ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.client.StreamCallbackWithID import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExecutorDiskUtils +import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -44,9 +47,10 @@ import org.apache.spark.util.Utils // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData(). private[spark] class IndexShuffleBlockResolver( conf: SparkConf, - _blockManager: BlockManager = null) + // var for testing + var _blockManager: BlockManager = null) extends ShuffleBlockResolver - with Logging { + with Logging with MigratableResolver { private lazy val blockManager = Option(_blockManager).getOrElse(SparkEnv.get.blockManager) @@ -55,6 +59,19 @@ private[spark] class IndexShuffleBlockResolver( def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None) + /** + * Get the shuffle files that are stored locally. Used for block migrations. + */ + override def getStoredShuffles(): Seq[ShuffleBlockInfo] = { + val allBlocks = blockManager.diskBlockManager.getAllBlocks() + allBlocks.flatMap { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + Some(ShuffleBlockInfo(shuffleId, mapId)) + case _ => + None + } + } + /** * Get the shuffle data file. * @@ -148,6 +165,82 @@ private[spark] class IndexShuffleBlockResolver( } } + /** + * Write a provided shuffle block as a stream. Used for block migrations. + * ShuffleBlockBatchIds must contain the full range represented in the ShuffleIndexBlock. + * Requires the caller to delete any shuffle index blocks where the shuffle block fails to + * put. + */ + override def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager): + StreamCallbackWithID = { + val file = blockId match { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + getIndexFile(shuffleId, mapId) + case ShuffleDataBlockId(shuffleId, mapId, _) => + getDataFile(shuffleId, mapId) + case _ => + throw new IllegalStateException(s"Unexpected shuffle block transfer ${blockId} as " + + s"${blockId.getClass().getSimpleName()}") + } + val fileTmp = Utils.tempFileWith(file) + val channel = Channels.newChannel( + serializerManager.wrapStream(blockId, + new FileOutputStream(fileTmp))) + + new StreamCallbackWithID { + + override def getID: String = blockId.name + + override def onData(streamId: String, buf: ByteBuffer): Unit = { + while (buf.hasRemaining) { + channel.write(buf) + } + } + + override def onComplete(streamId: String): Unit = { + logTrace(s"Done receiving shuffle block $blockId, now storing on local disk.") + channel.close() + val diskSize = fileTmp.length() + this.synchronized { + if (file.exists()) { + file.delete() + } + if (!fileTmp.renameTo(file)) { + throw new IOException(s"fail to rename file ${fileTmp} to ${file}") + } + } + blockManager.reportBlockStatus(blockId, BlockStatus(StorageLevel.DISK_ONLY, 0, diskSize)) + } + + override def onFailure(streamId: String, cause: Throwable): Unit = { + // the framework handles the connection itself, we just need to do local cleanup + logWarning(s"Error while uploading $blockId", cause) + channel.close() + fileTmp.delete() + } + } + } + + /** + * Get the index & data block for migration. + */ + def getMigrationBlocks(shuffleBlockInfo: ShuffleBlockInfo): List[(BlockId, ManagedBuffer)] = { + val shuffleId = shuffleBlockInfo.shuffleId + val mapId = shuffleBlockInfo.mapId + // Load the index block + val indexFile = getIndexFile(shuffleId, mapId) + val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + val indexFileSize = indexFile.length() + val indexBlockData = new FileSegmentManagedBuffer(transportConf, indexFile, 0, indexFileSize) + + // Load the data block + val dataFile = getDataFile(shuffleId, mapId) + val dataBlockId = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + val dataBlockData = new FileSegmentManagedBuffer(transportConf, dataFile, 0, dataFile.length()) + List((indexBlockId, indexBlockData), (dataBlockId, dataBlockData)) + } + + /** * Write an index file with the offsets of each block, plus a final offset at the end for the * end of the output file. This will be used by getBlockData to figure out where each block @@ -169,7 +262,7 @@ private[spark] class IndexShuffleBlockResolver( val dataFile = getDataFile(shuffleId, mapId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. - synchronized { + this.synchronized { val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) if (existingLengths != null) { // Another attempt for the same task has already written our map outputs successfully, diff --git a/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala new file mode 100644 index 0000000000000..3851fa6eca19b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala @@ -0,0 +1,48 @@ +/* + * 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.shuffle + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.client.StreamCallbackWithID +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.storage.BlockId + +/** + * :: Experimental :: + * An experimental trait to allow Spark to migrate shuffle blocks. + */ +@Experimental +@Since("3.1.0") +trait MigratableResolver { + /** + * Get the shuffle ids that are stored locally. Used for block migrations. + */ + def getStoredShuffles(): Seq[ShuffleBlockInfo] + + /** + * Write a provided shuffle block as a stream. Used for block migrations. + */ + def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager): + StreamCallbackWithID + + /** + * Get the blocks for migration for a particular shuffle and map. + */ + def getMigrationBlocks(shuffleBlockInfo: ShuffleBlockInfo): List[(BlockId, ManagedBuffer)] +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala new file mode 100644 index 0000000000000..99ceee81d349d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala @@ -0,0 +1,28 @@ +/* + * 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.shuffle + +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * An experimental case class used by MigratableResolver to return the shuffleId and mapId in a + * type safe way. + */ +@Experimental +case class ShuffleBlockInfo(shuffleId: Int, mapId: Long) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index f7b0e9b62fc29..7ae9117137caa 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.CPUS_PER_TASK import org.apache.spark.internal.config.Status._ @@ -868,13 +868,17 @@ private[spark] class AppStatusListener( // check if there is a new peak value for any of the executor level memory metrics // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed // for the live UI. - event.executorUpdates.foreach { case (_, peakUpdates) => + event.executorUpdates.foreach { case (key, peakUpdates) => liveExecutors.get(event.execId).foreach { exec => if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(peakUpdates)) { - maybeUpdate(exec, now) + update(exec, now) } } + + // Update stage level peak executor metrics. + updateStageLevelPeakExecutorMetrics(key._1, key._2, event.execId, peakUpdates, now) } + // Flush updates if necessary. Executor heartbeat is an event that happens periodically. Flush // here to ensure the staleness of Spark UI doesn't last more than // `max(heartbeat interval, liveUpdateMinFlushPeriod)`. @@ -885,17 +889,38 @@ private[spark] class AppStatusListener( } } - override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = { + override def onStageExecutorMetrics(event: SparkListenerStageExecutorMetrics): Unit = { val now = System.nanoTime() // check if there is a new peak value for any of the executor level memory metrics, // while reading from the log. SparkListenerStageExecutorMetrics are only processed // when reading logs. - liveExecutors.get(executorMetrics.execId).orElse( - deadExecutors.get(executorMetrics.execId)).foreach { exec => - if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { - update(exec, now) - } + liveExecutors.get(event.execId).orElse( + deadExecutors.get(event.execId)).foreach { exec => + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(event.executorMetrics)) { + update(exec, now) + } + } + + // Update stage level peak executor metrics. + updateStageLevelPeakExecutorMetrics( + event.stageId, event.stageAttemptId, event.execId, event.executorMetrics, now) + } + + private def updateStageLevelPeakExecutorMetrics( + stageId: Int, + stageAttemptId: Int, + executorId: String, + executorMetrics: ExecutorMetrics, + now: Long): Unit = { + Option(liveStages.get((stageId, stageAttemptId))).foreach { stage => + if (stage.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) { + update(stage, now) + } + val esummary = stage.executorSummary(executorId) + if (esummary.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) { + update(esummary, now) + } } } 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 0a8d188dc1553..5c6543fe28a18 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -506,7 +506,8 @@ private[spark] class AppStatusStore( tasks = Some(tasks), executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)), killedTasksSummary = stage.killedTasksSummary, - resourceProfileId = stage.resourceProfileId) + resourceProfileId = stage.resourceProfileId, + peakExecutorMetrics = stage.peakExecutorMetrics) } def rdd(rddId: Int): v1.RDDStorageInfo = { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 81478214994b0..0fadd330a01ad 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -365,6 +365,8 @@ private class LiveExecutorStageSummary( var metrics = createMetrics(default = 0L) + val peakExecutorMetrics = new ExecutorMetrics() + override protected def doUpdate(): Any = { val info = new v1.ExecutorStageSummary( taskTime, @@ -381,7 +383,8 @@ private class LiveExecutorStageSummary( metrics.shuffleWriteMetrics.recordsWritten, metrics.memoryBytesSpilled, metrics.diskBytesSpilled, - isBlacklisted) + isBlacklisted, + Some(peakExecutorMetrics).filter(_.isSet)) new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info) } @@ -420,6 +423,8 @@ private class LiveStage extends LiveEntity { var blackListedExecutors = new HashSet[String]() + val peakExecutorMetrics = new ExecutorMetrics() + // Used for cleanup of tasks after they reach the configured limit. Not written to the store. @volatile var cleaning = false var savedTasks = new AtomicInteger(0) @@ -484,7 +489,8 @@ private class LiveStage extends LiveEntity { tasks = None, executorSummary = None, killedTasksSummary = killedSummary, - resourceProfileId = info.resourceProfileId) + resourceProfileId = info.resourceProfileId, + Some(peakExecutorMetrics).filter(_.isSet)) } override protected def doUpdate(): Any = { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index e0c85fdf6fb5d..536a1fcd59cd0 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -19,7 +19,7 @@ package org.apache.spark.status.api.v1 import java.io.OutputStream import java.util.{List => JList} import java.util.zip.ZipOutputStream -import javax.ws.rs._ +import javax.ws.rs.{NotFoundException => _, _} import javax.ws.rs.core.{MediaType, Response, StreamingOutput} import scala.util.control.NonFatal @@ -104,10 +104,10 @@ private[v1] class AbstractApplicationResource extends BaseAppResource { val resourceProfileInfo = ui.store.resourceProfileInfo() new v1.ApplicationEnvironmentInfo( envInfo.runtime, - Utils.redact(ui.conf, envInfo.sparkProperties), - Utils.redact(ui.conf, envInfo.hadoopProperties), - Utils.redact(ui.conf, envInfo.systemProperties), - envInfo.classpathEntries, + Utils.redact(ui.conf, envInfo.sparkProperties).sortBy(_._1), + Utils.redact(ui.conf, envInfo.hadoopProperties).sortBy(_._1), + Utils.redact(ui.conf, envInfo.systemProperties).sortBy(_._1), + envInfo.classpathEntries.sortBy(_._1), resourceProfileInfo) } 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 05a7e96882d77..84bd430d9abef 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 @@ -17,7 +17,7 @@ package org.apache.spark.status.api.v1 import java.util.{HashMap, List => JList, Locale} -import javax.ws.rs._ +import javax.ws.rs.{NotFoundException => _, _} import javax.ws.rs.core.{Context, MediaType, MultivaluedMap, UriInfo} import org.apache.spark.ui.UIUtils diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index e89e29101a126..d207a6023f7f9 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -82,7 +82,10 @@ class ExecutorStageSummary private[spark]( val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long, - val isBlacklistedForStage: Boolean) + val isBlacklistedForStage: Boolean, + @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) + @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) + val peakMemoryMetrics: Option[ExecutorMetrics]) class ExecutorSummary private[spark]( val id: String, @@ -259,7 +262,10 @@ class StageData private[spark]( val tasks: Option[Map[Long, TaskData]], val executorSummary: Option[Map[String, ExecutorStageSummary]], val killedTasksSummary: Map[String, Int], - val resourceProfileId: Int) + val resourceProfileId: Int, + @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) + @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) + val peakExecutorMetrics: Option[ExecutorMetrics]) class TaskData private[spark]( val taskId: Long, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 68ed3aa5b062f..7b084e73c92f9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -38,7 +38,10 @@ sealed abstract class BlockId { // convenience methods def asRDDId: Option[RDDBlockId] = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None def isRDD: Boolean = isInstanceOf[RDDBlockId] - def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId] + def isShuffle: Boolean = { + (isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId] || + isInstanceOf[ShuffleDataBlockId] || isInstanceOf[ShuffleIndexBlockId]) + } def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId] override def toString: String = name diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6eec288015380..6ec93df67f7db 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -24,6 +24,7 @@ import java.nio.channels.Channels import java.util.Collections import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.HashMap import scala.concurrent.{ExecutionContext, Future} @@ -53,6 +54,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.serializer.{SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.{MigratableResolver, ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.shuffle.{ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock import org.apache.spark.storage.memory._ @@ -242,8 +244,8 @@ private[spark] class BlockManager( private var blockReplicationPolicy: BlockReplicationPolicy = _ - private var blockManagerDecommissioning: Boolean = false - private var decommissionManager: Option[BlockManagerDecommissionManager] = None + // This is volatile since if it's defined we should not accept remote blocks. + @volatile private var decommissioner: Option[BlockManagerDecommissioner] = None // A DownloadFileManager used to track all the files of remote blocks which are above the // specified memory threshold. Files will be deleted automatically based on weak reference. @@ -254,6 +256,15 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None + @inline final private def isDecommissioning() = { + decommissioner.isDefined + } + // This is a lazy val so someone can migrating RDDs even if they don't have a MigratableResolver + // for shuffles. Used in BlockManagerDecommissioner & block puts. + private[storage] lazy val migratableResolver: MigratableResolver = { + shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] + } + /** * Abstraction for storing blocks from bytes, whether they start in memory or on disk. * @@ -364,7 +375,7 @@ private[spark] class BlockManager( ThreadUtils.awaitReady(replicationFuture, Duration.Inf) } catch { case NonFatal(t) => - throw new Exception("Error occurred while waiting for replication to finish", t) + throw new SparkException("Error occurred while waiting for replication to finish", t) } } if (blockWasSuccessfullyStored) { @@ -617,6 +628,7 @@ private[spark] class BlockManager( */ override def getLocalBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { + logInfo(s"Getting local shuffle block ${blockId}") shuffleManager.shuffleBlockResolver.getBlockData(blockId) } else { getLocalBytes(blockId) match { @@ -650,6 +662,23 @@ private[spark] class BlockManager( blockId: BlockId, level: StorageLevel, classTag: ClassTag[_]): StreamCallbackWithID = { + + if (isDecommissioning()) { + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) + } + + if (blockId.isShuffle) { + logDebug(s"Putting shuffle block ${blockId}") + try { + return migratableResolver.putShuffleBlockAsStream(blockId, serializerManager) + } catch { + case e: ClassCastException => throw new SparkException( + s"Unexpected shuffle block ${blockId} with unsupported shuffle " + + s"resolver ${shuffleManager.shuffleBlockResolver}") + } + } + logDebug(s"Putting regular block ${blockId}") + // All other blocks val (_, tmpFile) = diskBlockManager.createTempLocalBlock() val channel = new CountingWritableChannel( Channels.newChannel(serializerManager.wrapForEncryption(new FileOutputStream(tmpFile)))) @@ -720,7 +749,7 @@ private[spark] class BlockManager( * it is still valid). This ensures that update in master will compensate for the increase in * memory on the storage endpoint. */ - private def reportBlockStatus( + private[spark] def reportBlockStatus( blockId: BlockId, status: BlockStatus, droppedMemorySize: Long = 0L): Unit = { @@ -1285,6 +1314,9 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + if (isDecommissioning()) { + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) + } val putBlockInfo = { val newInfo = new BlockInfo(level, classTag, tellMaster) @@ -1540,7 +1572,7 @@ private[spark] class BlockManager( /** * Get peer block managers in the system. */ - private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { + private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { peerFetchLock.synchronized { val cachedPeersTtl = conf.get(config.STORAGE_CACHED_PEERS_TTL) // milliseconds val diff = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lastPeerFetchTimeNs) @@ -1779,60 +1811,28 @@ private[spark] class BlockManager( blocksToRemove.size } - def decommissionBlockManager(): Unit = { - if (!blockManagerDecommissioning) { - logInfo("Starting block manager decommissioning process") - blockManagerDecommissioning = true - decommissionManager = Some(new BlockManagerDecommissionManager(conf)) - decommissionManager.foreach(_.start()) - } else { - logDebug("Block manager already in decommissioning state") + def decommissionBlockManager(): Unit = synchronized { + decommissioner match { + case None => + logInfo("Starting block manager decommissioning process...") + decommissioner = Some(new BlockManagerDecommissioner(conf, this)) + decommissioner.foreach(_.start()) + case Some(_) => + logDebug("Block manager already in decommissioning state") } } - /** - * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers - * Visible for testing + /* + * Returns the last migration time and a boolean denoting if all the blocks have been migrated. + * If there are any tasks running since that time the boolean may be incorrect. */ - def decommissionRddCacheBlocks(): Unit = { - val replicateBlocksInfo = master.getReplicateInfoForRDDBlocks(blockManagerId) - - if (replicateBlocksInfo.nonEmpty) { - logInfo(s"Need to replicate ${replicateBlocksInfo.size} blocks " + - "for block manager decommissioning") - } else { - logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") - return - } - - // Maximum number of storage replication failure which replicateBlock can handle - val maxReplicationFailures = conf.get( - config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) - - // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) - // so that we end up prioritize them over each other - val blocksFailedReplication = replicateBlocksInfo.map { - case ReplicateBlock(blockId, existingReplicas, maxReplicas) => - val replicatedSuccessfully = replicateBlock( - blockId, - existingReplicas.toSet, - maxReplicas, - maxReplicationFailures = Some(maxReplicationFailures)) - if (replicatedSuccessfully) { - logInfo(s"Block $blockId offloaded successfully, Removing block now") - removeBlock(blockId) - logInfo(s"Block $blockId removed") - } else { - logWarning(s"Failed to offload block $blockId") - } - (blockId, replicatedSuccessfully) - }.filterNot(_._2).map(_._1) - if (blocksFailedReplication.nonEmpty) { - logWarning("Blocks failed replication in cache decommissioning " + - s"process: ${blocksFailedReplication.mkString(",")}") - } + private[spark] def lastMigrationInfo(): (Long, Boolean) = { + decommissioner.map(_.lastMigrationInfo()).getOrElse((0, false)) } + private[storage] def getMigratableRDDBlocks(): Seq[ReplicateBlock] = + master.getReplicateInfoForRDDBlocks(blockManagerId) + /** * Remove all blocks belonging to the given broadcast. */ @@ -1901,58 +1901,8 @@ private[spark] class BlockManager( data.dispose() } - /** - * Class to handle block manager decommissioning retries - * It creates a Thread to retry offloading all RDD cache blocks - */ - private class BlockManagerDecommissionManager(conf: SparkConf) { - @volatile private var stopped = false - private val sleepInterval = conf.get( - config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) - - private val blockReplicationThread = new Thread { - override def run(): Unit = { - var failures = 0 - while (blockManagerDecommissioning - && !stopped - && !Thread.interrupted() - && failures < 20) { - try { - logDebug("Attempting to replicate all cached RDD blocks") - decommissionRddCacheBlocks() - logInfo("Attempt to replicate all cached blocks done") - Thread.sleep(sleepInterval) - } catch { - case _: InterruptedException => - logInfo("Interrupted during migration, will not refresh migrations.") - stopped = true - case NonFatal(e) => - failures += 1 - logError("Error occurred while trying to replicate cached RDD blocks" + - s" for block manager decommissioning (failure count: $failures)", e) - } - } - } - } - blockReplicationThread.setDaemon(true) - blockReplicationThread.setName("block-replication-thread") - - def start(): Unit = { - logInfo("Starting block replication thread") - blockReplicationThread.start() - } - - def stop(): Unit = { - if (!stopped) { - stopped = true - logInfo("Stopping block replication thread") - blockReplicationThread.interrupt() - } - } - } - def stop(): Unit = { - decommissionManager.foreach(_.stop()) + decommissioner.foreach(_.stop()) blockTransferService.close() if (blockStoreClient ne blockTransferService) { // Closing should be idempotent, but maybe not for the NioBlockTransferService. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala new file mode 100644 index 0000000000000..f0a8e47aa3200 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -0,0 +1,398 @@ +/* + * 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.storage + +import java.util.concurrent.ExecutorService +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.control.NonFatal + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.shuffle.{MigratableResolver, ShuffleBlockInfo} +import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock +import org.apache.spark.util.ThreadUtils + +/** + * Class to handle block manager decommissioning retries. + * It creates a Thread to retry offloading all RDD cache and Shuffle blocks + */ +private[storage] class BlockManagerDecommissioner( + conf: SparkConf, + bm: BlockManager) extends Logging { + + private val maxReplicationFailuresForDecommission = + conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) + + // Used for tracking if our migrations are complete. Readable for testing + @volatile private[storage] var lastRDDMigrationTime: Long = 0 + @volatile private[storage] var lastShuffleMigrationTime: Long = 0 + @volatile private[storage] var rddBlocksLeft: Boolean = true + @volatile private[storage] var shuffleBlocksLeft: Boolean = true + + /** + * This runnable consumes any shuffle blocks in the queue for migration. This part of a + * producer/consumer where the main migration loop updates the queue of blocks to be migrated + * periodically. On migration failure, the current thread will reinsert the block for another + * thread to consume. Each thread migrates blocks to a different particular executor to avoid + * distribute the blocks as quickly as possible without overwhelming any particular executor. + * + * There is no preference for which peer a given block is migrated to. + * This is notable different than the RDD cache block migration (further down in this file) + * which uses the existing priority mechanism for determining where to replicate blocks to. + * Generally speaking cache blocks are less impactful as they normally represent narrow + * transformations and we normally have less cache present than shuffle data. + * + * The producer/consumer model is chosen for shuffle block migration to maximize + * the chance of migrating all shuffle blocks before the executor is forced to exit. + */ + private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable { + @volatile var running = true + override def run(): Unit = { + var migrating: Option[(ShuffleBlockInfo, Int)] = None + logInfo(s"Starting migration thread for ${peer}") + // Once a block fails to transfer to an executor stop trying to transfer more blocks + try { + while (running && !Thread.interrupted()) { + migrating = Option(shufflesToMigrate.poll()) + migrating match { + case None => + logDebug("Nothing to migrate") + // Nothing to do right now, but maybe a transfer will fail or a new block + // will finish being committed. + val SLEEP_TIME_SECS = 1 + Thread.sleep(SLEEP_TIME_SECS * 1000L) + case Some((shuffleBlockInfo, retryCount)) => + if (retryCount < maxReplicationFailuresForDecommission) { + logInfo(s"Trying to migrate shuffle ${shuffleBlockInfo} to ${peer}") + val blocks = + bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo) + logDebug(s"Got migration sub-blocks ${blocks}") + blocks.foreach { case (blockId, buffer) => + logDebug(s"Migrating sub-block ${blockId}") + bm.blockTransferService.uploadBlockSync( + peer.host, + peer.port, + peer.executorId, + blockId, + buffer, + StorageLevel.DISK_ONLY, + null)// class tag, we don't need for shuffle + logDebug(s"Migrated sub block ${blockId}") + } + logDebug(s"Migrated ${shuffleBlockInfo} to ${peer}") + } else { + logError(s"Skipping block ${shuffleBlockInfo} because it has failed ${retryCount}") + } + numMigratedShuffles.incrementAndGet() + } + } + // This catch is intentionally outside of the while running block. + // if we encounter errors migrating to an executor we want to stop. + } catch { + case e: Exception => + migrating match { + case Some((shuffleMap, retryCount)) => + logError(s"Error during migration, adding ${shuffleMap} back to migration queue", e) + shufflesToMigrate.add((shuffleMap, retryCount + 1)) + case None => + logError(s"Error while waiting for block to migrate", e) + } + } + } + } + + // Shuffles which are either in queue for migrations or migrated + private val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]() + + // Shuffles which have migrated. This used to know when we are "done", being done can change + // if a new shuffle file is created by a running task. + private val numMigratedShuffles = new AtomicInteger(0) + + // Shuffles which are queued for migration & number of retries so far. + // Visible in storage for testing. + private[storage] val shufflesToMigrate = + new java.util.concurrent.ConcurrentLinkedQueue[(ShuffleBlockInfo, Int)]() + + // Set if we encounter an error attempting to migrate and stop. + @volatile private var stopped = false + @volatile private var stoppedRDD = + !conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED) + @volatile private var stoppedShuffle = + !conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED) + + private val migrationPeers = + mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]() + + private lazy val rddBlockMigrationExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission-rdd") + + private val rddBlockMigrationRunnable = new Runnable { + val sleepInterval = conf.get(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) + + override def run(): Unit = { + assert(conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) + while (!stopped && !stoppedRDD && !Thread.interrupted()) { + logInfo("Iterating on migrating from the block manager.") + // Validate we have peers to migrate to. + val peers = bm.getPeers(false) + // If we have no peers give up. + if (peers.isEmpty) { + stopped = true + stoppedRDD = true + } + try { + val startTime = System.nanoTime() + logDebug("Attempting to replicate all cached RDD blocks") + rddBlocksLeft = decommissionRddCacheBlocks() + lastRDDMigrationTime = startTime + logInfo("Attempt to replicate all cached blocks done") + logInfo(s"Waiting for ${sleepInterval} before refreshing migrations.") + Thread.sleep(sleepInterval) + } catch { + case e: InterruptedException => + logInfo("Interrupted during RDD migration, stopping") + stoppedRDD = true + case NonFatal(e) => + logError("Error occurred replicating RDD for block manager decommissioning.", + e) + stoppedRDD = true + } + } + } + } + + private lazy val shuffleBlockMigrationRefreshExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission-shuffle") + + private val shuffleBlockMigrationRefreshRunnable = new Runnable { + val sleepInterval = conf.get(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) + + override def run() { + assert(conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) + while (!stopped && !stoppedShuffle && !Thread.interrupted()) { + try { + logDebug("Attempting to replicate all shuffle blocks") + val startTime = System.nanoTime() + shuffleBlocksLeft = refreshOffloadingShuffleBlocks() + lastShuffleMigrationTime = startTime + logInfo("Done starting workers to migrate shuffle blocks") + Thread.sleep(sleepInterval) + } catch { + case e: InterruptedException => + logInfo("Interrupted during migration, will not refresh migrations.") + stoppedShuffle = true + case NonFatal(e) => + logError("Error occurred while trying to replicate for block manager decommissioning.", + e) + stoppedShuffle = true + } + } + } + } + + lazy val shuffleMigrationPool = ThreadUtils.newDaemonCachedThreadPool( + "migrate-shuffles", + conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_MAX_THREADS)) + + /** + * Tries to offload all shuffle blocks that are registered with the shuffle service locally. + * Note: this does not delete the shuffle files in-case there is an in-progress fetch + * but rather shadows them. + * Requires an Indexed based shuffle resolver. + * Note: if called in testing please call stopOffloadingShuffleBlocks to avoid thread leakage. + * Returns true if we are not done migrating shuffle blocks. + */ + private[storage] def refreshOffloadingShuffleBlocks(): Boolean = { + // Update the queue of shuffles to be migrated + logInfo("Offloading shuffle blocks") + val localShuffles = bm.migratableResolver.getStoredShuffles().toSet + val newShufflesToMigrate = localShuffles.diff(migratingShuffles).toSeq + shufflesToMigrate.addAll(newShufflesToMigrate.map(x => (x, 0)).asJava) + migratingShuffles ++= newShufflesToMigrate + + // Update the threads doing migrations + val livePeerSet = bm.getPeers(false).toSet + val currentPeerSet = migrationPeers.keys.toSet + val deadPeers = currentPeerSet.diff(livePeerSet) + val newPeers = livePeerSet.diff(currentPeerSet) + migrationPeers ++= newPeers.map { peer => + logDebug(s"Starting thread to migrate shuffle blocks to ${peer}") + val runnable = new ShuffleMigrationRunnable(peer) + shuffleMigrationPool.submit(runnable) + (peer, runnable) + } + // A peer may have entered a decommissioning state, don't transfer any new blocks + deadPeers.foreach { peer => + migrationPeers.get(peer).foreach(_.running = false) + } + // If we don't have anyone to migrate to give up + if (migrationPeers.values.find(_.running == true).isEmpty) { + stoppedShuffle = true + } + // If we found any new shuffles to migrate or otherwise have not migrated everything. + newShufflesToMigrate.nonEmpty || migratingShuffles.size < numMigratedShuffles.get() + } + + /** + * Stop migrating shuffle blocks. + */ + private[storage] def stopOffloadingShuffleBlocks(): Unit = { + logInfo("Stopping offloading shuffle blocks.") + // Stop as gracefully as possible. + migrationPeers.values.foreach{ _.running = false } + shuffleMigrationPool.shutdown() + shuffleMigrationPool.shutdownNow() + } + + /** + * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers + * Visible for testing + * Returns true if we have not migrated all of our RDD blocks. + */ + private[storage] def decommissionRddCacheBlocks(): Boolean = { + val replicateBlocksInfo = bm.getMigratableRDDBlocks() + // Refresh peers and validate we have somewhere to move blocks. + + if (replicateBlocksInfo.nonEmpty) { + logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + + "for block manager decommissioning") + } else { + logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") + return false + } + + // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) + // so that we end up prioritize them over each other + val blocksFailedReplication = replicateBlocksInfo.map { replicateBlock => + val replicatedSuccessfully = migrateBlock(replicateBlock) + (replicateBlock.blockId, replicatedSuccessfully) + }.filterNot(_._2).map(_._1) + if (blocksFailedReplication.nonEmpty) { + logWarning("Blocks failed replication in cache decommissioning " + + s"process: ${blocksFailedReplication.mkString(",")}") + return true + } + return false + } + + private def migrateBlock(blockToReplicate: ReplicateBlock): Boolean = { + val replicatedSuccessfully = bm.replicateBlock( + blockToReplicate.blockId, + blockToReplicate.replicas.toSet, + blockToReplicate.maxReplicas, + maxReplicationFailures = Some(maxReplicationFailuresForDecommission)) + if (replicatedSuccessfully) { + logInfo(s"Block ${blockToReplicate.blockId} offloaded successfully, Removing block now") + bm.removeBlock(blockToReplicate.blockId) + logInfo(s"Block ${blockToReplicate.blockId} removed") + } else { + logWarning(s"Failed to offload block ${blockToReplicate.blockId}") + } + replicatedSuccessfully + } + + def start(): Unit = { + logInfo("Starting block migration thread") + if (conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) { + rddBlockMigrationExecutor.submit(rddBlockMigrationRunnable) + } + if (conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) { + shuffleBlockMigrationRefreshExecutor.submit(shuffleBlockMigrationRefreshRunnable) + } + if (!conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED) && + !conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) { + logError(s"Storage decommissioning attempted but neither " + + s"${config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key} or " + + s"${config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key} is enabled ") + stopped = true + } + } + + def stop(): Unit = { + if (stopped) { + return + } else { + stopped = true + } + try { + rddBlockMigrationExecutor.shutdown() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + shuffleBlockMigrationRefreshExecutor.shutdown() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + stopOffloadingShuffleBlocks() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + logInfo("Forcing block migrations threads to stop") + try { + rddBlockMigrationExecutor.shutdownNow() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + shuffleBlockMigrationRefreshExecutor.shutdownNow() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + logInfo("Stopped storage decommissioner") + } + + /* + * Returns the last migration time and a boolean for if all blocks have been migrated. + * The last migration time is calculated to be the minimum of the last migration of any + * running migration (and if there are now current running migrations it is set to current). + * This provides a timeStamp which, if there have been no tasks running since that time + * we can know that all potential blocks that can be have been migrated off. + */ + private[storage] def lastMigrationInfo(): (Long, Boolean) = { + if (stopped || (stoppedRDD && stoppedShuffle)) { + // Since we don't have anything left to migrate ever (since we don't restart once + // stopped), return that we're done with a validity timestamp that doesn't expire. + (Long.MaxValue, true) + } else { + // Chose the min of the active times. See the function description for more information. + val lastMigrationTime = if (!stoppedRDD && !stoppedShuffle) { + Math.min(lastRDDMigrationTime, lastShuffleMigrationTime) + } else if (!stoppedShuffle) { + lastShuffleMigrationTime + } else { + lastRDDMigrationTime + } + + // Technically we could have blocks left if we encountered an error, but those blocks will + // never be migrated, so we don't care about them. + val blocksMigrated = (!shuffleBlocksLeft || stoppedShuffle) && (!rddBlocksLeft || stoppedRDD) + (lastMigrationTime, blocksMigrated) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 2a4817797a87c..a3d42348befaa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal import com.google.common.cache.CacheBuilder -import org.apache.spark.SparkConf +import org.apache.spark.{MapOutputTrackerMaster, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.shuffle.ExternalBlockStoreClient @@ -50,7 +50,8 @@ class BlockManagerMasterEndpoint( conf: SparkConf, listenerBus: LiveListenerBus, externalBlockStoreClient: Option[ExternalBlockStoreClient], - blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) + blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo], + mapOutputTracker: MapOutputTrackerMaster) extends IsolatedRpcEndpoint with Logging { // Mapping from executor id to the block manager's local disk directories. @@ -162,7 +163,8 @@ class BlockManagerMasterEndpoint( context.reply(true) case DecommissionBlockManagers(executorIds) => - decommissionBlockManagers(executorIds.flatMap(blockManagerIdByExecutor.get)) + val bmIds = executorIds.flatMap(blockManagerIdByExecutor.get) + decommissionBlockManagers(bmIds) context.reply(true) case GetReplicateInfoForRDDBlocks(blockManagerId) => @@ -539,6 +541,24 @@ class BlockManagerMasterEndpoint( storageLevel: StorageLevel, memSize: Long, diskSize: Long): Boolean = { + logDebug(s"Updating block info on master ${blockId} for ${blockManagerId}") + + if (blockId.isShuffle) { + blockId match { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + // Don't update the map output on just the index block + logDebug(s"Received shuffle index block update for ${shuffleId} ${mapId}, ignoring.") + return true + case ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) => + logDebug(s"Received shuffle data block update for ${shuffleId} ${mapId}, updating.") + mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) + return true + case _ => + logDebug(s"Unexpected shuffle block type ${blockId}" + + s"as ${blockId.getClass().getSimpleName()}") + return false + } + } if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala b/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala new file mode 100644 index 0000000000000..4684d9c67754d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala @@ -0,0 +1,21 @@ +/* + * 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.storage + +class BlockSavedOnDecommissionedBlockManagerException(blockId: BlockId) + extends Exception(s"Block $blockId cannot be saved on decommissioned executor") diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index f1962ef39fc06..2a3597e323543 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui -import java.net.{URI, URL} +import java.net.{URI, URL, URLDecoder} import java.util.EnumSet import javax.servlet.DispatcherType import javax.servlet.http._ @@ -377,8 +377,7 @@ private[spark] object JettyUtils extends Logging { if (baseRequest.isSecure) { return } - val httpsURI = createRedirectURI(scheme, baseRequest.getServerName, securePort, - baseRequest.getRequestURI, baseRequest.getQueryString) + val httpsURI = createRedirectURI(scheme, securePort, baseRequest) response.setContentLength(0) response.sendRedirect(response.encodeRedirectURL(httpsURI)) baseRequest.setHandled(true) @@ -440,16 +439,34 @@ private[spark] object JettyUtils extends Logging { handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) } + private def decodeURL(url: String, encoding: String): String = { + if (url == null) { + null + } else { + URLDecoder.decode(url, encoding) + } + } + // Create a new URI from the arguments, handling IPv6 host encoding and default ports. - private def createRedirectURI( - scheme: String, server: String, port: Int, path: String, query: String) = { + private def createRedirectURI(scheme: String, port: Int, request: Request): String = { + val server = request.getServerName val redirectServer = if (server.contains(":") && !server.startsWith("[")) { s"[${server}]" } else { server } val authority = s"$redirectServer:$port" - new URI(scheme, authority, path, query, null).toString + val queryEncoding = if (request.getQueryEncoding != null) { + request.getQueryEncoding + } else { + // By default decoding the URI as "UTF-8" should be enough for SparkUI + "UTF-8" + } + // The request URL can be raw or encoded here. To avoid the request URL being + // encoded twice, let's decode it here. + val requestURI = decodeURL(request.getRequestURI, queryEncoding) + val queryString = decodeURL(request.getQueryString, queryEncoding) + new URI(scheme, authority, requestURI, queryString, null).toString } def toVirtualHosts(connectors: String*): Array[String] = connectors.map("@" + _).toArray diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index bba5e3dda6c47..df239d6d0e187 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -255,7 +255,8 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP tasks = None, executorSummary = None, killedTasksSummary = Map(), - ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) + ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID, + peakExecutorMetrics = None) } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index ceaddb4306579..13f7cb453346f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -333,7 +333,7 @@ private[spark] object JsonProtocol { def accumulablesToJson(accumulables: Iterable[AccumulableInfo]): JArray = { JArray(accumulables .filterNot(_.name.exists(accumulableExcludeList.contains)) - .toList.map(accumulableInfoToJson)) + .toList.sortBy(_.id).map(accumulableInfoToJson)) } def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThreadRunner.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThreadRunner.scala new file mode 100644 index 0000000000000..18108aa819db9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThreadRunner.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.concurrent.Executors + +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration + +/** + * [[UninterruptibleThreadRunner]] ensures that all tasks are running in an + * [[UninterruptibleThread]]. A good example is Kafka consumer usage. + */ +private[spark] class UninterruptibleThreadRunner(threadName: String) { + private val thread = Executors.newSingleThreadExecutor((r: Runnable) => { + val t = new UninterruptibleThread(threadName) { + override def run(): Unit = { + r.run() + } + } + t.setDaemon(true) + t + }) + private val execContext = ExecutionContext.fromExecutorService(thread) + + def runUninterruptibly[T](body: => T): T = { + if (!Thread.currentThread.isInstanceOf[UninterruptibleThread]) { + val future = Future { + body + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } else { + body + } + } + + def shutdown(): Unit = { + thread.shutdown() + } +} diff --git a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json index 0b617a7d0aced..c2616129de954 100644 --- a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json @@ -5,283 +5,283 @@ "scalaVersion" : "version 2.11.8" }, "sparkProperties" : [ - [ "spark.blacklist.task.maxTaskAttemptsPerExecutor", "3" ], + [ "spark.app.id", "app-20161116163331-0000" ], + [ "spark.app.name", "Spark shell" ], + [ "spark.blacklist.application.maxFailedExecutorsPerNode", "2" ], + [ "spark.blacklist.application.maxFailedTasksPerExecutor", "1" ], [ "spark.blacklist.enabled", "TRUE" ], - [ "spark.driver.host", "172.22.0.167" ], + [ "spark.blacklist.stage.maxFailedExecutorsPerNode", "3" ], + [ "spark.blacklist.stage.maxFailedTasksPerExecutor", "3" ], + [ "spark.blacklist.task.maxTaskAttemptsPerExecutor", "3" ], [ "spark.blacklist.task.maxTaskAttemptsPerNode", "3" ], - [ "spark.eventLog.enabled", "TRUE" ], + [ "spark.blacklist.timeout", "1000000" ], + [ "spark.driver.host", "172.22.0.167" ], [ "spark.driver.port", "51459" ], - [ "spark.repl.class.uri", "spark://172.22.0.167:51459/classes" ], - [ "spark.jars", "" ], - [ "spark.repl.class.outputDir", "/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed" ], - [ "spark.app.name", "Spark shell" ], - [ "spark.blacklist.stage.maxFailedExecutorsPerNode", "3" ], - [ "spark.scheduler.mode", "FIFO" ], + [ "spark.eventLog.compress", "FALSE" ], + [ "spark.eventLog.dir", "/Users/jose/logs" ], + [ "spark.eventLog.enabled", "TRUE" ], [ "spark.eventLog.overwrite", "TRUE" ], - [ "spark.blacklist.stage.maxFailedTasksPerExecutor", "3" ], [ "spark.executor.id", "driver" ], - [ "spark.blacklist.application.maxFailedExecutorsPerNode", "2" ], - [ "spark.submit.deployMode", "client" ], - [ "spark.master", "local-cluster[4,4,1024]" ], [ "spark.home", "/Users/Jose/IdeaProjects/spark" ], - [ "spark.eventLog.dir", "/Users/jose/logs" ], + [ "spark.jars", "" ], + [ "spark.master", "local-cluster[4,4,1024]" ], + [ "spark.repl.class.outputDir", "/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed" ], + [ "spark.repl.class.uri", "spark://172.22.0.167:51459/classes" ], + [ "spark.scheduler.mode", "FIFO" ], [ "spark.sql.catalogImplementation", "in-memory" ], - [ "spark.eventLog.compress", "FALSE" ], - [ "spark.blacklist.application.maxFailedTasksPerExecutor", "1" ], - [ "spark.blacklist.timeout", "1000000" ], - [ "spark.app.id", "app-20161116163331-0000" ], + [ "spark.submit.deployMode", "client" ], [ "spark.task.maxFailures", "4" ] ], "hadoopProperties" : [ + [ "mapreduce.jobhistory.client.thread-count", "10" ], [ "mapreduce.jobtracker.address", "local" ], - [ "yarn.resourcemanager.scheduler.monitor.policies", "org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy" ], - [ "mapreduce.jobhistory.client.thread-count", "10" ] + [ "yarn.resourcemanager.scheduler.monitor.policies", "org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy" ] ], "systemProperties" : [ - [ "java.io.tmpdir", "/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/" ], - [ "line.separator", "\n" ], - [ "path.separator", ":" ], - [ "sun.management.compiler", "HotSpot 64-Bit Tiered Compilers" ], [ "SPARK_SUBMIT", "true" ], - [ "sun.cpu.endian", "little" ], - [ "java.specification.version", "1.8" ], - [ "java.vm.specification.name", "Java Virtual Machine Specification" ], - [ "java.vendor", "Oracle Corporation" ], - [ "java.vm.specification.version", "1.8" ], - [ "user.home", "/Users/Jose" ], + [ "awt.toolkit", "sun.lwawt.macosx.LWCToolkit" ], + [ "file.encoding", "UTF-8" ], [ "file.encoding.pkg", "sun.io" ], - [ "sun.nio.ch.bugLevel", "" ], + [ "file.separator", "/" ], [ "ftp.nonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], - [ "sun.arch.data.model", "64" ], - [ "sun.boot.library.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib" ], - [ "user.dir", "/Users/Jose/IdeaProjects/spark" ], - [ "java.library.path", "/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:." ], - [ "sun.cpu.isalist", "" ], - [ "os.arch", "x86_64" ], - [ "java.vm.version", "25.92-b14" ], + [ "gopherProxySet", "false" ], + [ "http.nonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], + [ "io.netty.maxDirectMemory", "0" ], + [ "java.awt.graphicsenv", "sun.awt.CGraphicsEnvironment" ], + [ "java.awt.printerjob", "sun.lwawt.macosx.CPrinterJob" ], + [ "java.class.version", "52.0" ], [ "java.endorsed.dirs", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed" ], - [ "java.runtime.version", "1.8.0_92-b14" ], - [ "java.vm.info", "mixed mode" ], [ "java.ext.dirs", "/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java" ], + [ "java.home", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre" ], + [ "java.io.tmpdir", "/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/" ], + [ "java.library.path", "/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:." ], [ "java.runtime.name", "Java(TM) SE Runtime Environment" ], - [ "file.separator", "/" ], - [ "io.netty.maxDirectMemory", "0" ], - [ "java.class.version", "52.0" ], - [ "scala.usejavacp", "true" ], + [ "java.runtime.version", "1.8.0_92-b14" ], [ "java.specification.name", "Java Platform API Specification" ], - [ "sun.boot.class.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes" ], - [ "file.encoding", "UTF-8" ], - [ "user.timezone", "America/Chicago" ], [ "java.specification.vendor", "Oracle Corporation" ], - [ "sun.java.launcher", "SUN_STANDARD" ], + [ "java.specification.version", "1.8" ], + [ "java.vendor", "Oracle Corporation" ], + [ "java.vendor.url", "http://java.oracle.com/" ], + [ "java.vendor.url.bug", "http://bugreport.sun.com/bugreport/" ], + [ "java.version", "1.8.0_92" ], + [ "java.vm.info", "mixed mode" ], + [ "java.vm.name", "Java HotSpot(TM) 64-Bit Server VM" ], + [ "java.vm.specification.name", "Java Virtual Machine Specification" ], + [ "java.vm.specification.vendor", "Oracle Corporation" ], + [ "java.vm.specification.version", "1.8" ], + [ "java.vm.vendor", "Oracle Corporation" ], + [ "java.vm.version", "25.92-b14" ], + [ "line.separator", "\n" ], + [ "os.arch", "x86_64" ], + [ "os.name", "Mac OS X" ], [ "os.version", "10.11.6" ], + [ "path.separator", ":" ], + [ "scala.usejavacp", "true" ], + [ "socksNonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], + [ "sun.arch.data.model", "64" ], + [ "sun.boot.class.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes" ], + [ "sun.boot.library.path", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib" ], + [ "sun.cpu.endian", "little" ], + [ "sun.cpu.isalist", "" ], + [ "sun.io.unicode.encoding", "UnicodeBig" ], + [ "sun.java.command", "org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala" ], + [ "sun.java.launcher", "SUN_STANDARD" ], + [ "sun.jnu.encoding", "UTF-8" ], + [ "sun.management.compiler", "HotSpot 64-Bit Tiered Compilers" ], + [ "sun.nio.ch.bugLevel", "" ], [ "sun.os.patch.level", "unknown" ], - [ "gopherProxySet", "false" ], - [ "java.vm.specification.vendor", "Oracle Corporation" ], [ "user.country", "US" ], - [ "sun.jnu.encoding", "UTF-8" ], - [ "http.nonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], + [ "user.dir", "/Users/Jose/IdeaProjects/spark" ], + [ "user.home", "/Users/Jose" ], [ "user.language", "en" ], - [ "socksNonProxyHosts", "local|*.local|169.254/16|*.169.254/16" ], - [ "java.vendor.url", "http://java.oracle.com/" ], - [ "java.awt.printerjob", "sun.lwawt.macosx.CPrinterJob" ], - [ "java.awt.graphicsenv", "sun.awt.CGraphicsEnvironment" ], - [ "awt.toolkit", "sun.lwawt.macosx.LWCToolkit" ], - [ "os.name", "Mac OS X" ], - [ "java.vm.vendor", "Oracle Corporation" ], - [ "java.vendor.url.bug", "http://bugreport.sun.com/bugreport/" ], [ "user.name", "jose" ], - [ "java.vm.name", "Java HotSpot(TM) 64-Bit Server VM" ], - [ "sun.java.command", "org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala" ], - [ "java.home", "/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre" ], - [ "java.version", "1.8.0_92" ], - [ "sun.io.unicode.encoding", "UnicodeBig" ] + [ "user.timezone", "America/Chicago" ] ], "classpathEntries" : [ - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/conf/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/core/target/jars/*", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], - [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar", "System Classpath" ] + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/conf/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/core/target/jars/*", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/", "System Classpath" ], + [ "/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes", "System Classpath" ] ], "resourceProfiles" : [ ] } diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index d2b3d1b069204..06015ec46e44d 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index d2b3d1b069204..06015ec46e44d 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 82489e94a84c8..8e6be68b4670c 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { @@ -28,19 +43,4 @@ "lastUpdatedEpoch" : 0, "endTimeEpoch" : 1562101355974 } ] -}, { - "id" : "application_1553914137147_0018", - "name" : "LargeBlocks", - "attempts" : [ { - "startTime" : "2019-04-08T20:39:44.286GMT", - "endTime" : "2019-04-08T20:40:46.454GMT", - "lastUpdated" : "", - "duration" : 62168, - "sparkUser" : "systest", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1554755984286, - "endTimeEpoch" : 1554756046454, - "lastUpdatedEpoch" : 0 - } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index ac2bb0e29b2fb..35d71f9d4409b 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { @@ -14,22 +29,20 @@ "lastUpdatedEpoch" : 0 } ] }, { - "id": "application_1555004656427_0144", - "name": "Spark shell", - "attempts": [ - { - "startTime": "2019-07-02T21:02:17.180GMT", - "endTime": "2019-07-02T21:02:35.974GMT", - "lastUpdated": "", - "duration": 18794, - "sparkUser": "tgraves", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "startTimeEpoch": 1562101337180, - "lastUpdatedEpoch": 0, - "endTimeEpoch": 1562101355974 - } - ] + "id" : "application_1555004656427_0144", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2019-07-02T21:02:17.180GMT", + "endTime" : "2019-07-02T21:02:35.974GMT", + "lastUpdated" : "", + "duration" : 18794, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1562101355974, + "startTimeEpoch" : 1562101337180, + "lastUpdatedEpoch" : 0 + } ] }, { "id" : "application_1553914137147_0018", "name" : "LargeBlocks", diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json index 156167606ff20..c6530b14bf271 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "app-20200706201101-0003", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-07-07T03:11:00.235GMT", + "endTime" : "2020-07-07T03:17:04.231GMT", + "lastUpdated" : "", + "duration" : 363996, + "sparkUser" : "terryk", + "completed" : true, + "appSparkVersion" : "3.1.0-SNAPSHOT", + "endTimeEpoch" : 1594091824231, + "startTimeEpoch" : 1594091460235, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1578436911597_0052", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json new file mode 100644 index 0000000000000..d455b97fdafa4 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json @@ -0,0 +1,204 @@ +[ { + "status" : "COMPLETE", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:21.040GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT", + "completionTime" : "2020-07-07T03:11:23.044GMT", + "executorDeserializeTime" : 3905, + "executorDeserializeCpuTime" : 979900000, + "executorRunTime" : 25579, + "executorCpuTime" : 8810338000, + "resultSize" : 33883, + "jvmGcTime" : 1010, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 384640, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "foreach at :26", + "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)", + "schedulingPool" : "default", + "rddIds" : [ 10, 8, 6, 7, 9 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } +}, { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:20.499GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:20.502GMT", + "completionTime" : "2020-07-07T03:11:20.930GMT", + "executorDeserializeTime" : 424, + "executorDeserializeCpuTime" : 63666000, + "executorRunTime" : 6105, + "executorCpuTime" : 426449000, + "resultSize" : 31546, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 16256, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266", + "description" : "broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)", + "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", + "schedulingPool" : "default", + "rddIds" : [ 5, 3, 4 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + } +}, { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:18.860GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:19.253GMT", + "completionTime" : "2020-07-07T03:11:20.381GMT", + "executorDeserializeTime" : 15399, + "executorDeserializeCpuTime" : 3668889000, + "executorRunTime" : 1292, + "executorCpuTime" : 172863000, + "resultSize" : 22375, + "jvmGcTime" : 1744, + "resultSerializationTime" : 3, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266", + "description" : "broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)", + "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", + "schedulingPool" : "default", + "rddIds" : [ 2, 0, 1 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 155100856, + "JVMOffHeapMemory" : 64239224, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 6964, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 6964, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 1852, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 7, + "MinorGCTime" : 33, + "MajorGCCount" : 3, + "MajorGCTime" : 110 + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json new file mode 100644 index 0000000000000..373510d23058e --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json @@ -0,0 +1,998 @@ +{ + "status" : "COMPLETE", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompleteTasks" : 16, + "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 16, + "submissionTime" : "2020-07-07T03:11:21.040GMT", + "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT", + "completionTime" : "2020-07-07T03:11:23.044GMT", + "executorDeserializeTime" : 3905, + "executorDeserializeCpuTime" : 979900000, + "executorRunTime" : 25579, + "executorCpuTime" : 8810338000, + "resultSize" : 33883, + "jvmGcTime" : 1010, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 384640, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, + "shuffleWriteRecords" : 0, + "name" : "foreach at :26", + "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)", + "schedulingPool" : "default", + "rddIds" : [ 10, 8, 6, 7, 9 ], + "accumulatorUpdates" : [ ], + "tasks" : { + "42" : { + "taskId" : 42, + "index" : 10, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.120GMT", + "duration" : 1923, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 229, + "executorDeserializeCpuTime" : 58152000, + "executorRunTime" : 1624, + "executorCpuTime" : 508230000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "37" : { + "taskId" : 37, + "index" : 5, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.100GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 256, + "executorDeserializeCpuTime" : 60890000, + "executorRunTime" : 1596, + "executorCpuTime" : 507192000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "46" : { + "taskId" : 46, + "index" : 14, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.132GMT", + "duration" : 1905, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 218, + "executorDeserializeCpuTime" : 51464000, + "executorRunTime" : 1618, + "executorCpuTime" : 490927000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 69, + "gettingResultTime" : 0 + }, + "38" : { + "taskId" : 38, + "index" : 6, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.104GMT", + "duration" : 1835, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 255, + "executorDeserializeCpuTime" : 60358000, + "executorRunTime" : 1498, + "executorCpuTime" : 414110000, + "resultSize" : 2158, + "jvmGcTime" : 62, + "resultSerializationTime" : 11, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "33" : { + "taskId" : 33, + "index" : 1, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.082GMT", + "duration" : 1943, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 267, + "executorDeserializeCpuTime" : 54442000, + "executorRunTime" : 1597, + "executorCpuTime" : 519178000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "41" : { + "taskId" : 41, + "index" : 9, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.116GMT", + "duration" : 1916, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 240, + "executorDeserializeCpuTime" : 55787000, + "executorRunTime" : 1614, + "executorCpuTime" : 489923000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 62, + "gettingResultTime" : 0 + }, + "32" : { + "taskId" : 32, + "index" : 0, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.077GMT", + "duration" : 1960, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 271, + "executorDeserializeCpuTime" : 56827000, + "executorRunTime" : 1619, + "executorCpuTime" : 496683000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 70, + "gettingResultTime" : 0 + }, + "34" : { + "taskId" : 34, + "index" : 2, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.087GMT", + "duration" : 1939, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 265, + "executorDeserializeCpuTime" : 69492000, + "executorRunTime" : 1606, + "executorCpuTime" : 508433000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 68, + "gettingResultTime" : 0 + }, + "45" : { + "taskId" : 45, + "index" : 13, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.129GMT", + "duration" : 1895, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 221, + "executorDeserializeCpuTime" : 54222000, + "executorRunTime" : 1595, + "executorCpuTime" : 495138000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 79, + "gettingResultTime" : 0 + }, + "44" : { + "taskId" : 44, + "index" : 12, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.126GMT", + "duration" : 1917, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 222, + "executorDeserializeCpuTime" : 51988000, + "executorRunTime" : 1624, + "executorCpuTime" : 498187000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + }, + "39" : { + "taskId" : 39, + "index" : 7, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.109GMT", + "duration" : 1915, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 254, + "executorDeserializeCpuTime" : 64380000, + "executorRunTime" : 1596, + "executorCpuTime" : 539451000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 65, + "gettingResultTime" : 0 + }, + "35" : { + "taskId" : 35, + "index" : 3, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.091GMT", + "duration" : 1925, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 263, + "executorDeserializeCpuTime" : 62944000, + "executorRunTime" : 1598, + "executorCpuTime" : 502908000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "43" : { + "taskId" : 43, + "index" : 11, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.123GMT", + "duration" : 1906, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 225, + "executorDeserializeCpuTime" : 48849000, + "executorRunTime" : 1609, + "executorCpuTime" : 502120000, + "resultSize" : 2115, + "jvmGcTime" : 66, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 72, + "gettingResultTime" : 0 + }, + "40" : { + "taskId" : 40, + "index" : 8, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.112GMT", + "duration" : 1904, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 246, + "executorDeserializeCpuTime" : 69760000, + "executorRunTime" : 1595, + "executorCpuTime" : 510597000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 + }, + "36" : { + "taskId" : 36, + "index" : 4, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.095GMT", + "duration" : 1920, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 260, + "executorDeserializeCpuTime" : 112849000, + "executorRunTime" : 1596, + "executorCpuTime" : 503010000, + "resultSize" : 2115, + "jvmGcTime" : 62, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 64, + "gettingResultTime" : 0 + }, + "47" : { + "taskId" : 47, + "index" : 15, + "attempt" : 0, + "launchTime" : "2020-07-07T03:11:21.136GMT", + "duration" : 1878, + "executorId" : "0", + "host" : "127.0.0.1", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 213, + "executorDeserializeCpuTime" : 47496000, + "executorRunTime" : 1594, + "executorCpuTime" : 1324251000, + "resultSize" : 2115, + "jvmGcTime" : 52, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 24040, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } + }, + "executorLogs" : { + "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout", + "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr" + }, + "schedulerDelay" : 71, + "gettingResultTime" : 0 + } + }, + "executorSummary" : { + "0" : { + "taskTime" : 30596, + "failedTasks" : 0, + "succeededTasks" : 16, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + } + }, + "driver" : { + "taskTime" : 0, + "failedTasks" : 0, + "succeededTasks" : 0, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false, + "peakMemoryMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } + } + }, + "killedTasksSummary" : { }, + "resourceProfileId" : 0, + "peakExecutorMetrics" : { + "JVMHeapMemory" : 213367864, + "JVMOffHeapMemory" : 189011656, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 2133349, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 2133349, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 282024, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 13, + "MinorGCTime" : 115, + "MajorGCCount" : 4, + "MajorGCTime" : 339 + } +} diff --git a/core/src/test/resources/spark-events/app-20200706201101-0003 b/core/src/test/resources/spark-events/app-20200706201101-0003 new file mode 100644 index 0000000000000..b2923ca0f001f --- /dev/null +++ b/core/src/test/resources/spark-events/app-20200706201101-0003 @@ -0,0 +1,124 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.1.0-SNAPSHOT"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":64415},"Maximum Memory":384093388,"Timestamp":1594091461118,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","Java Version":"1.8.0_231 (Oracle Corporation)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","yarn.timeline-service.timeline-client.number-of-async-entities-to-merge":"10","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","yarn.nodemanager.collector-service.address":"${yarn.nodemanager.hostname}:8048","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.resourcemanager.placement-constraints.handler":"disabled","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"1000","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.resourcemanager.placement-constraints.algorithm.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm.DefaultPlacementAlgorithm","yarn.sharedcache.webapp.address":"0.0.0.0:8788","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.nodemanager.resource.memory.enforced":"true","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","fs.s3a.etag.checksum.enabled":"false","yarn.nodemanager.container-metrics.enable":"true","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.scheduler.configuration.leveldb-store.path":"${hadoop.tmp.dir}/yarn/system/confstore","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","fs.s3a.committer.name":"file","mapreduce.task.profile.reduces":"0-2","hadoop.zk.num-retries":"1000","yarn.webapp.xfs-filter.enabled":"true","seq.io.sort.mb":"100","yarn.scheduler.configuration.max.version":"100","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.groups.shell.command.timeout":"0s","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.nodemanager.elastic-memory-control.oom-handler":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.DefaultOOMHandler","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.abfss.impl":"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.epoch.range":"0","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","fs.AbstractFileSystem.wasbs.impl":"org.apache.hadoop.fs.azure.Wasbs","yarn.timeline-service.entity-group-fs-store.cache-store-class":"org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.router.webapp.https.address":"0.0.0.0:8091","yarn.nodemanager.webapp.cross-origin.enabled":"false","fs.wasb.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem","yarn.resourcemanager.auto-update.containers":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.scheduler.configuration.zk-store.parent-path":"/confstore","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","yarn.app.attempt.diagnostics.limit.kc":"64","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.AbstractFileSystem.abfs.impl":"org.apache.hadoop.fs.azurebfs.Abfs","yarn.timeline-service.writer.flush-interval-seconds":"60","fs.s3a.fast.upload.active.blocks":"4","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","yarn.nodemanager.log-aggregation.num-log-files-per-app":"30","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","fs.s3a.committer.magic.enabled":"false","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","yarn.system-metrics-publisher.enabled":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","hadoop.tags.system":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","yarn.scheduler.configuration.store.max-logs":"1000","yarn.nodemanager.node-attributes.provider.fetch-interval-ms":"600000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","hadoop.zk.acl":"world:anyone:rwcda","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.node-attribute.fs-store.impl.class":"org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore","fs.s3a.retry.interval":"500ms","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","yarn.nodemanager.elastic-memory-control.timeout-sec":"5","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","yarn.timeline-service.reader.webapp.address":"${yarn.timeline-service.webapp.address}","yarn.resourcemanager.placement-constraints.algorithm.pool-size":"1","yarn.timeline-service.hbase.coprocessor.jar.hdfs.location":"/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","fs.client.resolve.topology.enabled":"false","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","yarn.timeline-service.webapp.xfs-filter.xframe-options":"SAMEORIGIN","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","hadoop.registry.zk.session.timeout.ms":"60000","yarn.federation.cache-ttl.secs":"300","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.resourcemanager.opportunistic-container-allocation.nodes-used":"10","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","fs.AbstractFileSystem.abfss.impl":"org.apache.hadoop.fs.azurebfs.Abfss","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","yarn.resourcemanager.delegation-token.max-conf-size-bytes":"*********(redacted)","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","yarn.router.clientrm.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.clientrm.DefaultClientRequestInterceptor","yarn.resourcemanager.zk-appid-node.split-index":"0","ftp.blocksize":"67108864","yarn.nodemanager.runtime.linux.sandbox-mode.local-dirs.permissions":"read","yarn.router.rmadmin.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.rmadmin.DefaultRMAdminRequestInterceptor","yarn.nodemanager.log-container-debug-info.enabled":"true","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","yarn.nodemanager.container-retry-minimum-interval-ms":"1000","hadoop.security.groups.cache.secs":"300","yarn.federation.enabled":"false","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","yarn.nodemanager.amrmproxy.ha.enable":"false","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","yarn.client.application-client-protocol.poll-interval-ms":"200","yarn.scheduler.configuration.leveldb-store.compaction-interval-secs":"86400","yarn.timeline-service.writer.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.resourcemanager.placement-constraints.scheduler.pool-size":"1","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","yarn.nodemanager.resource-plugins.gpu.docker-plugin":"nvidia-docker-v1","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","yarn.scheduler.configuration.store.class":"file","yarn.resourcemanager.nm-container-queuing.queue-limit-stdev":"1.0f","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ipc.client.bind.wildcard.addr":"false","yarn.resourcemanager.webapp.rest-csrf.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","yarn.nodemanager.runtime.linux.docker.allowed-container-networks":"host,none,bridge","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.jobhistory.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.rm.system-metrics-publisher.emit-container-events":"false","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","fs.s3a.s3guard.ddb.throttle.retry.interval":"100ms","fs.AbstractFileSystem.wasb.impl":"org.apache.hadoop.fs.azure.Wasb","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.runtime.linux.docker.userremapping-gid-threshold":"1","yarn.nodemanager.webapp.rest-csrf.enabled":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","hadoop.security.key.default.bitlength":"128","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","yarn.webapp.filter-entity-list-by-user":"false","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","io.erasurecode.codec.rs.rawcoders":"rs_native,rs_java","io.erasurecode.codec.rs-legacy.rawcoders":"rs-legacy_java","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.resourcemanager.placement-constraints.algorithm.iterator":"SERIAL","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","mapreduce.job.cache.limit.max-resources-mb":"0","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"5000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.nodemanager.container.stderr.pattern":"{*stderr*,*STDERR*}","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","io.erasurecode.codec.xor.rawcoders":"xor_native,xor_java","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","yarn.federation.state-store.class":"org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"-1","yarn.resourcemanager.placement-constraints.retry-attempts":"3","hadoop.caller.context.enabled":"false","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","hadoop.zk.timeout-ms":"10000","fs.s3a.s3guard.cli.prune.age":"86400000","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.nodemanager.runtime.linux.sandbox-mode":"disabled","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3a.committer.threads":"8","hadoop.zk.retry-interval-ms":"1000","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","yarn.nodemanager.resource.memory.enabled":"false","fs.azure.authorization.caching.enable":"true","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"true","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","io.map.index.interval":"128","yarn.resourcemanager.nm-container-queuing.max-queue-wait-time-ms":"100","fs.abfs.impl":"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem","mapreduce.job.counters.max":"120","mapreduce.jobhistory.webapp.rest-csrf.enabled":"false","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","hadoop.security.auth_to_local.mechanism":"hadoop","yarn.timeline-service.app-collector.linger-period.ms":"60000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","yarn.nodemanager.runtime.linux.docker.enable-userremapping.allowed":"true","yarn.webapp.api-service.enable":"false","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","fs.ftp.impl":"org.apache.hadoop.fs.ftp.FTPFileSystem","yarn.nodemanager.container.stderr.tail.bytes":"4096","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","mapreduce.outputcommitter.factory.scheme.s3a":"org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory","hadoop.security.groups.cache.background.reload":"false","yarn.nodemanager.container-monitor.enabled":"true","yarn.nodemanager.elastic-memory-control.enabled":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","mapreduce.fileoutputcommitter.algorithm.version":"2","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","fs.s3a.retry.throttle.limit":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","yarn.nodemanager.resource-monitor.interval-ms":"3000","yarn.nodemanager.resource-plugins.gpu.allowed-gpu-devices":"auto","mapreduce.job.sharedcache.mode":"disabled","yarn.nodemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.shuffle.listen.queue.size":"128","yarn.scheduler.configuration.mutation.acl-policy.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy","mapreduce.map.cpu.vcores":"1","yarn.log-aggregation.file-formats":"TFile","yarn.timeline-service.client.fd-retain-secs":"300","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","yarn.router.webapp.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.webapp.DefaultRequestInterceptorREST","yarn.nodemanager.resource.memory.cgroups.soft-limit-percentage":"90.0","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.nodemanager.disk-validator":"basic","yarn.app.mapreduce.client.job.max-retries":"3","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","yarn.nodemanager.container-diagnostics-maximum-size":"10000","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","fs.s3a.assumed.role.session.duration":"30m","hadoop.security.group.mapping.ldap.conversion.rule":"none","hadoop.ssl.server.conf":"ssl-server.xml","fs.s3a.retry.throttle.interval":"1000ms","seq.io.sort.factor":"100","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","yarn.nodemanager.opportunistic-containers-use-pause-for-preemption":"false","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.timeline-service.reader.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.nodemanager.runtime.linux.docker.userremapping-uid-threshold":"1","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","mapreduce.job.cache.limit.max-single-resource-mb":"0","yarn.nodemanager.runtime.linux.docker.stop.grace-period":"10","yarn.resourcemanager.resource-profiles.source-file":"resource-profiles.json","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.jobhistory.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","yarn.timeline-service.client.drain-entities.timeout.ms":"2000","yarn.nodemanager.resource-plugins.fpga.vendor-plugin.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","mapreduce.job.encrypted-intermediate-data.buffer.kb":"128","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","mapreduce.job.encrypted-intermediate-data":"false","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","mapreduce.jobhistory.webapp.xfs-filter.xframe-options":"SAMEORIGIN","yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled":"false","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.timeline-service.reader.webapp.https.address":"${yarn.timeline-service.webapp.https.address}","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.local-fs.single-disk-limit.check.kill-limit-exceed":"true","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"0","ipc.maximum.response.length":"134217728","yarn.resourcemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","yarn.nodemanager.runtime.linux.docker.host-pid-namespace.allowed":"false","yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed":"false","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","fs.s3a.max.total.tasks":"5","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","yarn.resourcemanager.decommissioning-nodes-watcher.poll-interval-secs":"20","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.timeline-service.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.ifile.readahead.bytes":"4194304","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","hadoop.security.group.mapping.ldap.search.group.hierarchy.levels":"0","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","yarn.resourcemanager.application-timeouts.monitor.interval-ms":"3000","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.nodemanager.resource-plugins.fpga.allowed-fpga-devices":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.nodemanager.logaggregation.threadpool-size-max":"100","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"-1","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","yarn.router.interceptor.user.threadpool-size":"5","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","fs.s3a.socket.recv.buffer":"8192","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","mapreduce.job.heap.memory-mb.ratio":"0.8","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","yarn.resourcemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","yarn.scheduler.configuration.fs.path":"file://${hadoop.tmp.dir}/yarn/system/schedconf","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"-1","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","mapreduce.fileoutputcommitter.task.cleanup.enabled":"false","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","fs.azure.saskey.usecontainersaskeyforallaccess":"true","yarn.sharedcache.nm.uploader.thread-count":"20","yarn.resourcemanager.nodemanager-graceful-decommission-timeout-secs":"3600","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","yarn.nodemanager.opportunistic-containers-max-queue-length":"0","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","yarn.resourcemanager.opportunistic-container-allocation.enabled":"false","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","hadoop.system.tags":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.log-aggregation.file-controller.TFile.class":"org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.resourcemanager.nm-container-queuing.min-queue-length":"5","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"100M","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","mapreduce.job.encrypted-intermediate-data-key-size-bits":"128","fs.s3a.metadatastore.authoritative":"false","yarn.nodemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","ha.health-monitor.sleep-after-disconnect.ms":"1000","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"binary","yarn.resourcemanager.ha.enabled":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.jobhistory.intermediate-user-done-dir.permissions":"770","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.nodemanager.runtime.linux.docker.default-container-network":"host","yarn.router.webapp.address":"0.0.0.0:8089","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.nodemanager.resource-plugins.gpu.docker-plugin.nvidia-docker-v1.endpoint":"http://localhost:3476/v1.0/docker/cli","yarn.app.mapreduce.am.container.log.limit.kb":"0","ipc.client.connect.retry.interval":"1000","yarn.timeline-service.http-cross-origin.enabled":"false","fs.wasbs.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure","yarn.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms":"1000","fs.s3a.committer.staging.tmp.path":"tmp/staging","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","yarn.resourcemanager.nm-container-queuing.min-queue-wait-time-ms":"10","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","yarn.resourcemanager.zk-max-znode-size.bytes":"1048576","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","fs.s3a.committer.staging.conflict-mode":"fail","mapreduce.client.libjars.wildcard":"true","fs.s3a.committer.staging.unique-filenames":"true","yarn.nodemanager.node-attributes.provider.fetch-timeout-ms":"1200000","fs.s3a.list.version":"2","ftp.client-write-packet-size":"65536","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","hadoop.security.key.default.cipher":"AES/CTR/NoPadding","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","mapreduce.job.local-fs.single-disk-limit.check.interval-ms":"5000","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8049","ipc.server.listen.queue.size":"128","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","fs.s3a.assumed.role.credentials.provider":"org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.numa-awareness.enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","yarn.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","yarn.client.nodemanager-connect.retry-interval-ms":"10000","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME,HADOOP_HOME,PATH,LANG,TZ","yarn.sharedcache.nested-level":"3","yarn.timeline-service.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","fs.azure.user.agent.prefix":"unknown","yarn.resourcemanager.zk-delegation-token-node.split-index":"*********(redacted)","yarn.nodemanager.numa-awareness.read-topology":"false","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","yarn.registry.class":"org.apache.hadoop.registry.client.impl.FSRegistryOperationsService","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","yarn.scheduler.queue-placement-rules":"user-group","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"System Properties":{"java.io.tmpdir":"/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib","user.dir":"*********(redacted)","java.library.path":"*********(redacted)/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.231-b11","jetty.git.hash":"ab228fde9e55e9164c738d7fa121f8ac5acd51c9","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_231-b11","java.vm.info":"mixed mode","java.ext.dirs":"*********(redacted)/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.15.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","jdk.lang.Process.launchMechanism":"POSIX_SPAWN","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"terryk","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master spark://*********(redacted) --conf spark.eventLog.logStageExecutorMetrics=true --conf spark.eventLog.enabled=true --class org.apache.spark.repl.Main --name Spark shell spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","java.version":"1.8.0_231","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"*********(redacted)/audience-annotations-0.5.0.jar":"System Classpath","*********(redacted)/zstd-jni-1.4.5-2.jar":"System Classpath","*********(redacted)/hadoop-yarn-common-3.2.0.jar":"System Classpath","*********(redacted)/apache-spark/common/unsafe/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-identity-1.0.1.jar":"System Classpath","*********(redacted)/jersey-hk2-2.30.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/jars/*":"System Classpath","*********(redacted)/slf4j-log4j12-1.7.30.jar":"System Classpath","*********(redacted)/kerby-pkix-1.0.1.jar":"System Classpath","*********(redacted)/metrics-jmx-4.1.1.jar":"System Classpath","*********(redacted)/xz-1.5.jar":"System Classpath","*********(redacted)/jakarta.validation-api-2.0.2.jar":"System Classpath","*********(redacted)/jetty-webapp-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/metrics-core-4.1.1.jar":"System Classpath","*********(redacted)/apache-spark/graphx/target/scala-2.12/classes/":"System Classpath","*********(redacted)/metrics-graphite-4.1.1.jar":"System Classpath","*********(redacted)/jersey-server-2.30.jar":"System Classpath","*********(redacted)/jetty-jndi-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-proxy-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/parquet-jackson-1.10.1.jar":"System Classpath","*********(redacted)/commons-crypto-1.0.0.jar":"System Classpath","*********(redacted)/xbean-asm7-shaded-4.15.jar":"System Classpath","*********(redacted)/spark-kvstore_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/netty-3.10.6.Final.jar":"System Classpath","*********(redacted)/jsr305-3.0.2.jar":"System Classpath","*********(redacted)/osgi-resource-locator-1.0.3.jar":"System Classpath","*********(redacted)/kerb-server-1.0.1.jar":"System Classpath","*********(redacted)/avro-1.8.2.jar":"System Classpath","*********(redacted)/jersey-media-jaxb-2.30.jar":"System Classpath","*********(redacted)/accessors-smart-1.2.jar":"System Classpath","*********(redacted)/spark-network-shuffle_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/protobuf-java-2.5.0.jar":"System Classpath","*********(redacted)/minlog-1.3.0.jar":"System Classpath","*********(redacted)/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","*********(redacted)/jetty-continuation-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/sql/hive/target/scala-2.12/classes/":"System Classpath","*********(redacted)/commons-net-3.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/jars/*":"System Classpath","*********(redacted)/istack-commons-runtime-3.0.8.jar":"System Classpath","*********(redacted)/jsp-api-2.1.jar":"System Classpath","*********(redacted)/hadoop-annotations-3.2.0.jar":"System Classpath","*********(redacted)/spark-launcher_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-servlets-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-hdfs-client-3.2.0.jar":"System Classpath","*********(redacted)/hive-storage-api-2.7.1.jar":"System Classpath","*********(redacted)/aopalliance-repackaged-2.6.1.jar":"System Classpath","*********(redacted)/kerby-xdr-1.0.1.jar":"System Classpath","*********(redacted)/orc-mapreduce-1.5.10.jar":"System Classpath","*********(redacted)/jackson-databind-2.10.0.jar":"System Classpath","*********(redacted)/spark-mllib_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","*********(redacted)/hk2-utils-2.6.1.jar":"System Classpath","*********(redacted)/jetty-security-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-client-3.2.0.jar":"System Classpath","*********(redacted)/hadoop-auth-3.2.0.jar":"System Classpath","*********(redacted)/gson-2.2.4.jar":"System Classpath","*********(redacted)/htrace-core4-4.1.0-incubating.jar":"System Classpath","*********(redacted)/curator-client-2.12.0.jar":"System Classpath","*********(redacted)/kerby-util-1.0.1.jar":"System Classpath","*********(redacted)/jackson-jaxrs-base-2.9.5.jar":"System Classpath","*********(redacted)/jackson-module-paranamer-2.10.0.jar":"System Classpath","*********(redacted)/commons-beanutils-1.9.3.jar":"System Classpath","*********(redacted)/woodstox-core-5.0.3.jar":"System Classpath","*********(redacted)/arrow-memory-0.15.1.jar":"System Classpath","*********(redacted)/activation-1.1.1.jar":"System Classpath","*********(redacted)/okhttp-2.7.5.jar":"System Classpath","*********(redacted)/jackson-mapper-asl-1.9.13.jar":"System Classpath","*********(redacted)/core-1.1.2.jar":"System Classpath","*********(redacted)/jetty-plus-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/resource-managers/yarn/target/scala-2.12/classes/":"System Classpath","*********(redacted)/antlr4-runtime-4.7.1.jar":"System Classpath","*********(redacted)/commons-compress-1.8.1.jar":"System Classpath","*********(redacted)/spark-tags_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/orc-shims-1.5.10.jar":"System Classpath","*********(redacted)/opencsv-2.3.jar":"System Classpath","*********(redacted)/compress-lzf-1.0.3.jar":"System Classpath","*********(redacted)/jackson-core-asl-1.9.13.jar":"System Classpath","*********(redacted)/objenesis-2.5.1.jar":"System Classpath","*********(redacted)/apache-spark/conf/":"System Classpath","*********(redacted)/apache-spark/common/network-common/target/scala-2.12/classes/":"System Classpath","*********(redacted)/httpcore-4.4.4.jar":"System Classpath","*********(redacted)/jcip-annotations-1.0-1.jar":"System Classpath","*********(redacted)/apache-spark/launcher/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/resource-managers/mesos/target/scala-2.12/classes":"System Classpath","*********(redacted)/apache-spark/sql/hive-thriftserver/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jetty-util-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-server-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-xml-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","*********(redacted)/javax.servlet-api-3.1.0.jar":"System Classpath","*********(redacted)/jackson-core-2.10.0.jar":"System Classpath","*********(redacted)/jetty-client-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/netty-all-4.1.47.Final.jar":"System Classpath","*********(redacted)/arpack_combined_all-0.1.jar":"System Classpath","*********(redacted)/jersey-container-servlet-core-2.30.jar":"System Classpath","*********(redacted)/paranamer-2.8.jar":"System Classpath","*********(redacted)/spark-sql_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-common-3.2.0.jar":"System Classpath","*********(redacted)/jersey-common-2.30.jar":"System Classpath","*********(redacted)/hk2-locator-2.6.1.jar":"System Classpath","*********(redacted)/okio-1.6.0.jar":"System Classpath","*********(redacted)/jetty-http-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","*********(redacted)/ivy-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/scala-2.12/classes/":"System Classpath","*********(redacted)/scala-library-2.12.10.jar":"System Classpath","*********(redacted)/scala-xml_2.12-1.2.0.jar":"System Classpath","*********(redacted)/apache-spark/streaming/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spark-repl_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/apache-spark/common/tags/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerby-config-1.0.1.jar":"System Classpath","*********(redacted)/zookeeper-3.4.14.jar":"System Classpath","*********(redacted)/janino-3.1.2.jar":"System Classpath","*********(redacted)/breeze_2.12-1.0.jar":"System Classpath","*********(redacted)/hk2-api-2.6.1.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","*********(redacted)/flatbuffers-java-1.9.0.jar":"System Classpath","*********(redacted)/curator-framework-2.13.0.jar":"System Classpath","*********(redacted)/metrics-jvm-4.1.1.jar":"System Classpath","*********(redacted)/stax2-api-3.1.4.jar":"System Classpath","*********(redacted)/json-smart-2.3.jar":"System Classpath","*********(redacted)/parquet-hadoop-1.10.1.jar":"System Classpath","*********(redacted)/kerb-common-1.0.1.jar":"System Classpath","*********(redacted)/breeze-macros_2.12-1.0.jar":"System Classpath","*********(redacted)/scala-compiler-2.12.10.jar":"System Classpath","*********(redacted)/guava-14.0.1.jar":"System Classpath","*********(redacted)/jul-to-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/spark-assembly_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-core_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-graphx_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","*********(redacted)/spark-unsafe_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-io-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/JTransforms-3.1.jar":"System Classpath","*********(redacted)/commons-io-2.5.jar":"System Classpath","*********(redacted)/spark-catalyst_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spire_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-cli-1.2.jar":"System Classpath","*********(redacted)/shims-0.7.45.jar":"System Classpath","*********(redacted)/spotbugs-annotations-3.1.9.jar":"System Classpath","*********(redacted)/kerb-simplekdc-1.0.1.jar":"System Classpath","*********(redacted)/commons-codec-1.11.jar":"System Classpath","*********(redacted)/jcl-over-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/lz4-java-1.7.1.jar":"System Classpath","*********(redacted)/aopalliance-1.0.jar":"System Classpath","*********(redacted)/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","*********(redacted)/json4s-jackson_2.12-3.6.6.jar":"System Classpath","*********(redacted)/commons-math3-3.4.1.jar":"System Classpath","*********(redacted)/shapeless_2.12-2.3.3.jar":"System Classpath","*********(redacted)/spark-mllib-local_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/dnsjava-2.1.7.jar":"System Classpath","*********(redacted)/unused-1.0.0.jar":"System Classpath","*********(redacted)/log4j-1.2.17.jar":"System Classpath","*********(redacted)/kerb-client-1.0.1.jar":"System Classpath","*********(redacted)/commons-lang-2.6.jar":"System Classpath","*********(redacted)/apache-spark/repl/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-util-1.0.1.jar":"System Classpath","*********(redacted)/slf4j-api-1.7.30.jar":"System Classpath","*********(redacted)/jackson-annotations-2.10.0.jar":"System Classpath","*********(redacted)/kerb-core-1.0.1.jar":"System Classpath","*********(redacted)/apache-spark/sql/catalyst/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/apache-spark/examples/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jakarta.annotation-api-1.3.5.jar":"System Classpath","*********(redacted)/json4s-scalap_2.12-3.6.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/arrow-format-0.15.1.jar":"System Classpath","*********(redacted)/kryo-shaded-4.0.2.jar":"System Classpath","*********(redacted)/scala-reflect-2.12.10.jar":"System Classpath","*********(redacted)/kerb-admin-1.0.1.jar":"System Classpath","*********(redacted)/hadoop-yarn-client-3.2.0.jar":"System Classpath","*********(redacted)/nimbus-jose-jwt-4.41.1.jar":"System Classpath","*********(redacted)/apache-spark/common/kvstore/target/scala-2.12/classes/":"System Classpath","*********(redacted)/oro-2.0.8.jar":"System Classpath","*********(redacted)/jakarta.inject-2.6.1.jar":"System Classpath","*********(redacted)/chill-java-0.9.5.jar":"System Classpath","*********(redacted)/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","*********(redacted)/jersey-container-servlet-2.30.jar":"System Classpath","*********(redacted)/py4j-0.10.9.jar":"System Classpath","*********(redacted)/parquet-format-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/sql/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jline-2.14.6.jar":"System Classpath","*********(redacted)/JLargeArrays-1.5.jar":"System Classpath","*********(redacted)/kerby-asn1-1.0.1.jar":"System Classpath","*********(redacted)/jaxb-runtime-2.3.2.jar":"System Classpath","*********(redacted)/pmml-model-1.4.8.jar":"System Classpath","*********(redacted)/parquet-encoding-1.10.1.jar":"System Classpath","*********(redacted)/machinist_2.12-0.6.8.jar":"System Classpath","*********(redacted)/commons-compiler-3.1.2.jar":"System Classpath","*********(redacted)/kerb-crypto-1.0.1.jar":"System Classpath","*********(redacted)/aircompressor-0.10.jar":"System Classpath","*********(redacted)/leveldbjni-all-1.8.jar":"System Classpath","*********(redacted)/metrics-json-4.1.1.jar":"System Classpath","*********(redacted)/jackson-module-jaxb-annotations-2.9.5.jar":"System Classpath","*********(redacted)/jaxb-api-2.2.11.jar":"System Classpath","*********(redacted)/spire-util_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/jetty-servlet-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/spark-network-common_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-streaming_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/json4s-core_2.12-3.6.6.jar":"System Classpath","*********(redacted)/stream-2.9.6.jar":"System Classpath","*********(redacted)/spark-sketch_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/chill_2.12-0.9.5.jar":"System Classpath","*********(redacted)/commons-configuration2-2.1.1.jar":"System Classpath","*********(redacted)/univocity-parsers-2.8.3.jar":"System Classpath","*********(redacted)/parquet-common-1.10.1.jar":"System Classpath","*********(redacted)/threeten-extra-1.5.0.jar":"System Classpath","*********(redacted)/token-provider-1.0.1.jar":"System Classpath","*********(redacted)/commons-text-1.6.jar":"System Classpath","*********(redacted)/apache-spark/common/sketch/target/scala-2.12/classes/":"System Classpath","*********(redacted)/javax.inject-1.jar":"System Classpath","*********(redacted)/httpclient-4.5.2.jar":"System Classpath","*********(redacted)/avro-ipc-1.8.2.jar":"System Classpath","*********(redacted)/curator-recipes-2.13.0.jar":"System Classpath","*********(redacted)/commons-collections-3.2.2.jar":"System Classpath","*********(redacted)/jersey-client-2.30.jar":"System Classpath","*********(redacted)/snappy-java-1.1.7.5.jar":"System Classpath","*********(redacted)/algebra_2.12-2.0.0-M2.jar":"System Classpath","*********(redacted)/javassist-3.25.0-GA.jar":"System Classpath","*********(redacted)/arrow-vector-0.15.1.jar":"System Classpath","*********(redacted)/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","*********(redacted)/re2j-1.1.jar":"System Classpath","*********(redacted)/parquet-column-1.10.1.jar":"System Classpath","*********(redacted)/pyrolite-4.30.jar":"System Classpath","*********(redacted)/apache-spark/common/network-shuffle/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/common/network-yarn/target/scala-2.12/classes":"System Classpath","*********(redacted)/hadoop-yarn-api-3.2.0.jar":"System Classpath","*********(redacted)/macro-compat_2.12-1.1.1.jar":"System Classpath","*********(redacted)/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-lang3-3.10.jar":"System Classpath","*********(redacted)/json4s-ast_2.12-3.6.6.jar":"System Classpath","*********(redacted)/RoaringBitmap-0.7.45.jar":"System Classpath","*********(redacted)/orc-core-1.5.10.jar":"System Classpath","*********(redacted)/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20200706201101-0003","Timestamp":1594091460235,"User":"terryk"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1594091463318,"Executor ID":"0","Executor Info":{"Host":"127.0.0.1","Total Cores":16,"Log Urls":{"stdout":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout","stderr":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"127.0.0.1","Port":64419},"Maximum Memory":384093388,"Timestamp":1594091463413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"foreach at :26","details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","physicalPlanDescription":"== Physical Plan ==\nDeserializeToObject (8)\n+- * BroadcastHashJoin Inner BuildLeft (7)\n :- BroadcastExchange (5)\n : +- * BroadcastHashJoin Inner BuildRight (4)\n : :- * LocalTableScan (1)\n : +- BroadcastExchange (3)\n : +- LocalTableScan (2)\n +- * LocalTableScan (6)\n\n\n(1) LocalTableScan [codegen id : 1]\nOutput [3]: [i1#10, j1#11, k1#12]\nArguments: [i1#10, j1#11, k1#12]\n\n(2) LocalTableScan\nOutput [3]: [i2#26, j2#27, k2#28]\nArguments: [i2#26, j2#27, k2#28]\n\n(3) BroadcastExchange\nInput [3]: [i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]\n\n(4) BroadcastHashJoin [codegen id : 1]\nLeft keys [1]: [i1#10]\nRight keys [1]: [i2#26]\nJoin condition: None\n\n(5) BroadcastExchange\nInput [6]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]\n\n(6) LocalTableScan\nOutput [3]: [i3#42, j3#43, k3#44]\nArguments: [i3#42, j3#43, k3#44]\n\n(7) BroadcastHashJoin [codegen id : 2]\nLeft keys [1]: [i2#26]\nRight keys [1]: [i3#42]\nJoin condition: None\n\n(8) DeserializeToObject\nInput [9]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28, i3#42, j3#43, k3#44]\nArguments: createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row\n\n","sparkPlanInfo":{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i2#26], [i3#42], Inner, BuildLeft","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i1#10], [i2#26], Inner, BuildRight","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i1#10, j1#11, k1#12]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":8,"metricType":"sum"}]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i2#26, j2#27, k2#28]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":13,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":9,"metricType":"size"},{"name":"time to collect","accumulatorId":10,"metricType":"timing"},{"name":"time to build","accumulatorId":11,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":12,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":7,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":6,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"time to collect","accumulatorId":3,"metricType":"timing"},{"name":"time to build","accumulatorId":4,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":5,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i3#42, j3#43, k3#44]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":0,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1594091478577} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1594091478844,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1594091479253,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1594091479271,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1594091479271,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480364,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":109,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":1387,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8922000,"Value":8922000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":80,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235295000,"Value":235295000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":962,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":235295000,"Executor Run Time":80,"Executor CPU Time":8922000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":218,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":2778,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8860000,"Value":17782000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":81,"Value":161,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":213308000,"Value":448603000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":1925,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":213308000,"Executor Run Time":81,"Executor CPU Time":8860000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":327,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":4165,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10463000,"Value":28245000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":240,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":228677000,"Value":677280000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":2887,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":228677000,"Executor Run Time":79,"Executor CPU Time":10463000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":436,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":5555,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8782000,"Value":37027000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":323,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196368000,"Value":873648000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":3849,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196368000,"Executor Run Time":83,"Executor CPU Time":8782000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1594091479253,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":545,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":6939,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10820000,"Value":47847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":400,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":221708000,"Value":1095356000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":4811,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":221708000,"Executor Run Time":77,"Executor CPU Time":10820000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":654,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1406,"Value":8345,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":13213000,"Value":61060000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":78,"Value":478,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":260380000,"Value":1355736000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":5774,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":260380000,"Executor Run Time":78,"Executor CPU Time":13213000,"Peak Execution Memory":0,"Result Size":1406,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":763,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":9736,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9913000,"Value":70973000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":76,"Value":554,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":210788000,"Value":1566524000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":6736,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":210788000,"Executor Run Time":76,"Executor CPU Time":9913000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":872,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":11127,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8784000,"Value":79757000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":633,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235620000,"Value":1802144000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":964,"Value":7700,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":964,"Executor Deserialize CPU Time":235620000,"Executor Run Time":79,"Executor CPU Time":8784000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":981,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":12518,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12053000,"Value":91810000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":719,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":215398000,"Value":2017542000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":8663,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":215398000,"Executor Run Time":86,"Executor CPU Time":12053000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1090,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1403,"Value":13921,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9030000,"Value":100840000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":798,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196266000,"Value":2213808000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":9625,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196266000,"Executor Run Time":79,"Executor CPU Time":9030000,"Peak Execution Memory":0,"Result Size":1403,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1199,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":15311,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10087000,"Value":110927000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":877,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":195342000,"Value":2409150000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":10587,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":195342000,"Executor Run Time":79,"Executor CPU Time":10087000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1308,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1411,"Value":16722,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12920000,"Value":123847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":963,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":236044000,"Value":2645194000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":961,"Value":11548,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":961,"Executor Deserialize CPU Time":236044000,"Executor Run Time":86,"Executor CPU Time":12920000,"Peak Execution Memory":0,"Result Size":1411,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1417,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1430,"Value":18152,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10478000,"Value":134325000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":1040,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":205925000,"Value":2851119000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":12510,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":205925000,"Executor Run Time":77,"Executor CPU Time":10478000,"Peak Execution Memory":0,"Result Size":1430,"JVM GC Time":109,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1526,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":19539,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8972000,"Value":143297000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":87,"Value":1127,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":206247000,"Value":3057366000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":13473,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":206247000,"Executor Run Time":87,"Executor CPU Time":8972000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1635,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1452,"Value":20991,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":20898000,"Value":164195000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":1210,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":389356000,"Value":3446722000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":14436,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":389356000,"Executor Run Time":83,"Executor CPU Time":20898000,"Peak Execution Memory":0,"Result Size":1452,"JVM GC Time":109,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1744,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8668000,"Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":82,"Value":1292,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":222167000,"Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":15399,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":222167000,"Executor Run Time":82,"Executor CPU Time":8668000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Completion Time":1594091480381,"Accumulables":[{"ID":17,"Name":"internal.metrics.executorRunTime","Value":1292,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Value":1744,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"number of output rows","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Value":15399,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1594091480385,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[9,1048632],[10,1736],[11,37],[12,5]]} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1594091480498,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":1016,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":1966,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3116000,"Value":3116000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":378,"Value":378,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4258000,"Value":4258000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":4258000,"Executor Run Time":378,"Executor CPU Time":3116000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":2032,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1951,"Value":3917,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2895000,"Value":6011000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":379,"Value":757,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2641000,"Value":6899000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":58,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":2641000,"Executor Run Time":379,"Executor CPU Time":2895000,"Peak Execution Memory":1016,"Result Size":1951,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"54","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":3048,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":5874,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4164000,"Value":10175000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1137,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2012000,"Value":8911000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":86,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2012000,"Executor Run Time":380,"Executor CPU Time":4164000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"80","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"72","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":4064,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":7831,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4285000,"Value":14460000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1517,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2749000,"Value":11660000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":112,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":2749000,"Executor Run Time":380,"Executor CPU Time":4285000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480924,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"93","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"31","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":5080,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2007,"Value":9838,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":16921000,"Value":31381000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":1900,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3854000,"Value":15514000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":138,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":3854000,"Executor Run Time":383,"Executor CPU Time":16921000,"Peak Execution Memory":1016,"Result Size":2007,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480925,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"120","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"111","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":6096,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":11793,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4798000,"Value":36179000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2282,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2959000,"Value":18473000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":162,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":2959000,"Executor Run Time":382,"Executor CPU Time":4798000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480926,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"140","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"132","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":7112,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2015,"Value":13808,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4850000,"Value":41029000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2664,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4278000,"Value":22751000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":187,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4278000,"Executor Run Time":382,"Executor CPU Time":4850000,"Peak Execution Memory":1016,"Result Size":2015,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"161","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"153","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"51","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":8128,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1998,"Value":15806,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4100000,"Value":45129000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3047,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3622000,"Value":26373000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":215,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":3622000,"Executor Run Time":383,"Executor CPU Time":4100000,"Peak Execution Memory":1016,"Result Size":1998,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"182","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"57","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":9144,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1973,"Value":17779,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4350000,"Value":49479000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3430,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3471000,"Value":29844000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":240,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":3471000,"Executor Run Time":383,"Executor CPU Time":4350000,"Peak Execution Memory":1016,"Result Size":1973,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"202","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":10160,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":19745,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4682000,"Value":54161000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":384,"Value":3814,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2652000,"Value":32496000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":23,"Value":263,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":23,"Executor Deserialize CPU Time":2652000,"Executor Run Time":384,"Executor CPU Time":4682000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"223","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"210","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":11176,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2008,"Value":21753,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3954000,"Value":58115000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":4196,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4289000,"Value":36785000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":288,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4289000,"Executor Run Time":382,"Executor CPU Time":3954000,"Peak Execution Memory":1016,"Result Size":2008,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"243","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"228","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"76","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":12192,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1978,"Value":23731,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":349926000,"Value":408041000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":385,"Value":4581,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":14543000,"Value":51328000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":315,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":14543000,"Executor Run Time":385,"Executor CPU Time":349926000,"Peak Execution Memory":1016,"Result Size":1978,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"263","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"246","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"82","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":13208,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1965,"Value":25696,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4473000,"Value":412514000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":4962,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3906000,"Value":55234000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":342,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":3906000,"Executor Run Time":381,"Executor CPU Time":4473000,"Peak Execution Memory":1016,"Result Size":1965,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"283","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"264","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":14224,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":27651,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":6459000,"Value":418973000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":5343,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3298000,"Value":58532000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":366,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":3298000,"Executor Run Time":381,"Executor CPU Time":6459000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"304","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"282","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"94","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":15240,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1954,"Value":29605,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4365000,"Value":423338000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":5723,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2411000,"Value":60943000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":394,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2411000,"Executor Run Time":380,"Executor CPU Time":4365000,"Peak Execution Memory":1016,"Result Size":1954,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480930,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":16256,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1941,"Value":31546,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3111000,"Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":6105,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2723000,"Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":30,"Value":424,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":30,"Executor Deserialize CPU Time":2723000,"Executor Run Time":382,"Executor CPU Time":3111000,"Peak Execution Memory":1016,"Result Size":1941,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Completion Time":1594091480930,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"number of output rows","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":44,"Name":"internal.metrics.resultSize","Value":31546,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Value":16256,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Value":424,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"number of output rows","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.executorCpuTime","Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"duration","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":42,"Name":"internal.metrics.executorRunTime","Value":6105,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1594091480930,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[2,1048616],[3,2276],[4,13],[5,2]]} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1594091481039,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[2],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091482939,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1467","Value":"1467","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"6250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":24040,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2158,"Value":2158,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":414110000,"Value":414110000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1498,"Value":1498,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60358000,"Value":60358000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":255,"Value":255,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":255,"Executor Deserialize CPU Time":60358000,"Executor Run Time":1498,"Executor CPU Time":414110000,"Peak Execution Memory":24040,"Result Size":2158,"JVM GC Time":62,"Result Serialization Time":11,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483014,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"3030","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"12500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":48080,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":52,"Value":114,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":4273,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":1324251000,"Value":1738361000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1594,"Value":3092,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":47496000,"Value":107854000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":213,"Value":468,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":213,"Executor Deserialize CPU Time":47496000,"Executor Run Time":1594,"Executor CPU Time":1324251000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":52,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"4593","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"18750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":72120,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":176,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":6388,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":507192000,"Value":2245553000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":4688,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60890000,"Value":168744000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":256,"Value":724,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":256,"Executor Deserialize CPU Time":60890000,"Executor Run Time":1596,"Executor CPU Time":507192000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"6157","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"25000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":96160,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":238,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":8503,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":503010000,"Value":2748563000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":6284,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":112849000,"Value":281593000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":260,"Value":984,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":260,"Executor Deserialize CPU Time":112849000,"Executor Run Time":1596,"Executor CPU Time":503010000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"7723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"31250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":120200,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":300,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":10618,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502908000,"Value":3251471000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1598,"Value":7882,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":62944000,"Value":344537000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":263,"Value":1247,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":263,"Executor Deserialize CPU Time":62944000,"Executor Run Time":1598,"Executor CPU Time":502908000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1562","Value":"9285","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"37500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":144240,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":362,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":12733,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":510597000,"Value":3762068000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":9477,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69760000,"Value":414297000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":246,"Value":1493,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":246,"Executor Deserialize CPU Time":69760000,"Executor Run Time":1595,"Executor CPU Time":510597000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"10849","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"43750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":168280,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":14848,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":495138000,"Value":4257206000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":11072,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54222000,"Value":468519000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":221,"Value":1714,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":221,"Executor Deserialize CPU Time":54222000,"Executor Run Time":1595,"Executor CPU Time":495138000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"12415","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"50000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":192320,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":486,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":16963,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":539451000,"Value":4796657000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":12668,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":64380000,"Value":532899000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":254,"Value":1968,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":254,"Executor Deserialize CPU Time":64380000,"Executor Run Time":1596,"Executor CPU Time":539451000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483025,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"13981","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"56250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":216360,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":548,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":19078,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":519178000,"Value":5315835000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1597,"Value":14265,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54442000,"Value":587341000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":267,"Value":2235,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":267,"Executor Deserialize CPU Time":54442000,"Executor Run Time":1597,"Executor CPU Time":519178000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483026,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1574","Value":"15555","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"62500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":240400,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":614,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":21193,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508433000,"Value":5824268000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1606,"Value":15871,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69492000,"Value":656833000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":265,"Value":2500,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":265,"Executor Deserialize CPU Time":69492000,"Executor Run Time":1606,"Executor CPU Time":508433000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483029,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1578","Value":"17133","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"68750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":264440,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":680,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":23308,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502120000,"Value":6326388000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1609,"Value":17480,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":48849000,"Value":705682000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":225,"Value":2725,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":225,"Executor Deserialize CPU Time":48849000,"Executor Run Time":1609,"Executor CPU Time":502120000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483032,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1583","Value":"18716","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"75000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":288480,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":746,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":25423,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":489923000,"Value":6816311000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1614,"Value":19094,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":55787000,"Value":761469000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":240,"Value":2965,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":240,"Executor Deserialize CPU Time":55787000,"Executor Run Time":1614,"Executor CPU Time":489923000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"20303","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"81250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":312520,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":812,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":27538,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":490927000,"Value":7307238000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1618,"Value":20712,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51464000,"Value":812933000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":218,"Value":3183,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":218,"Executor Deserialize CPU Time":51464000,"Executor Run Time":1618,"Executor CPU Time":490927000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"21890","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"87500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":336560,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":878,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":29653,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":496683000,"Value":7803921000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1619,"Value":22331,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":56827000,"Value":869760000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":271,"Value":3454,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":271,"Executor Deserialize CPU Time":56827000,"Executor Run Time":1619,"Executor CPU Time":496683000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"23483","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"93750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":360600,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":944,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":31768,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508230000,"Value":8312151000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":23955,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":58152000,"Value":927912000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":229,"Value":3683,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":229,"Executor Deserialize CPU Time":58152000,"Executor Run Time":1624,"Executor CPU Time":508230000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":384640,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":1010,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":33883,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":498187000,"Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":25579,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51988000,"Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":222,"Value":3905,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":222,"Executor Deserialize CPU Time":51988000,"Executor Run Time":1624,"Executor CPU Time":498187000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":213367864,"JVMOffHeapMemory":189011656,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":2133349,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":2133349,"OffHeapUnifiedMemory":0,"DirectPoolMemory":282024,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":13,"MinorGCTime":115,"MajorGCCount":4,"MajorGCTime":339}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at :26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at :26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at :26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at :26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.(:34)\n$line19.$read$$iw$$iw$$iw$$iw.(:36)\n$line19.$read$$iw$$iw$$iw.(:38)\n$line19.$read$$iw$$iw.(:40)\n$line19.$read$$iw.(:42)\n$line19.$read.(:44)\n$line19.$read$.(:48)\n$line19.$read$.()\n$line19.$eval$.$print$lzycompute(:7)\n$line19.$eval$.$print(:6)\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Completion Time":1594091483044,"Accumulables":[{"ID":68,"Name":"internal.metrics.executorCpuTime","Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Value":11,"Internal":true,"Count Failed Values":true},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Value":384640,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Value":3905,"Internal":true,"Count Failed Values":true},{"ID":14,"Name":"number of output rows","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.executorRunTime","Value":25579,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Value":1010,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":69,"Name":"internal.metrics.resultSize","Value":33883,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"duration","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1594091483044,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1594091483045} +{"Event":"SparkListenerApplicationEnd","Timestamp":1594091824231} diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index d701cb65460af..e4dfa149a7d20 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -25,8 +25,8 @@ import scala.collection.mutable.ArrayBuffer import scala.ref.WeakReference import scala.util.control.NonFatal -import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException +import org.scalatest.matchers.must.Matchers import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer diff --git a/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala b/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala index 435b927068e60..1ba13c2ef1897 100644 --- a/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala +++ b/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala @@ -19,9 +19,12 @@ package org.apache.spark import scala.concurrent.duration._ +import org.apache.spark.TestUtils.createTempScriptWithExpectedOutput import org.apache.spark.internal.config._ import org.apache.spark.rdd.{PartitionPruningRDD, RDD} +import org.apache.spark.resource.TestResourceIDs.{EXECUTOR_GPU_ID, TASK_GPU_ID, WORKER_GPU_ID} import org.apache.spark.scheduler.BarrierJobAllocationFailed._ +import org.apache.spark.scheduler.BarrierJobSlotsNumberCheckFailed import org.apache.spark.util.ThreadUtils /** @@ -259,4 +262,37 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext testSubmitJob(sc, rdd, message = ERROR_MESSAGE_BARRIER_REQUIRE_MORE_SLOTS_THAN_CURRENT_TOTAL_NUMBER) } + + test("SPARK-32518: CoarseGrainedSchedulerBackend.maxNumConcurrentTasks should " + + "consider all kinds of resources for the barrier stage") { + withTempDir { dir => + val discoveryScript = createTempScriptWithExpectedOutput( + dir, "gpuDiscoveryScript", """{"name": "gpu","addresses":["0"]}""") + + val conf = new SparkConf() + // Setup a local cluster which would only has one executor with 2 CPUs and 1 GPU. + .setMaster("local-cluster[1, 2, 1024]") + .setAppName("test-cluster") + .set(WORKER_GPU_ID.amountConf, "1") + .set(WORKER_GPU_ID.discoveryScriptConf, discoveryScript) + .set(EXECUTOR_GPU_ID.amountConf, "1") + .set(TASK_GPU_ID.amountConf, "1") + // disable barrier stage retry to fail the application as soon as possible + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES, 1) + sc = new SparkContext(conf) + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) + + val exception = intercept[BarrierJobSlotsNumberCheckFailed] { + // Setup a barrier stage which contains 2 tasks and each task requires 1 CPU and 1 GPU. + // Therefore, the total resources requirement (2 CPUs and 2 GPUs) of this barrier stage + // can not be satisfied since the cluster only has 2 CPUs and 1 GPU in total. + sc.parallelize(Range(1, 10), 2) + .barrier() + .mapPartitions { iter => iter } + .collect() + } + assert(exception.getMessage.contains("[SPARK-24819]: Barrier execution " + + "mode does not allow run a barrier stage that requires more slots")) + } + } } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 27862806c0840..3478b16325dd6 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark import org.scalatest.Assertions._ -import org.scalatest.Matchers import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} +import org.scalatest.matchers.must.Matchers import org.scalatest.time.{Millis, Span} import org.apache.spark.internal.config diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 8037f4a9447dd..5b367d2fb01d4 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -21,15 +21,15 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import org.mockito.ArgumentMatchers.{any, eq => meq} -import org.mockito.Mockito.{mock, never, times, verify, when} +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito._ import org.scalatest.PrivateMethodTester import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, ResourceProfileManager, TaskResourceRequests} +import org.apache.spark.resource._ import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -501,6 +501,175 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(numExecutorsToAddForDefaultProfile(manager) === 1) } + test("SPARK-31418: one stage being unschedulable") { + val clock = new ManualClock() + val conf = createConf(0, 5, 0).set(config.EXECUTOR_CORES, 2) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(createStageInfo(0, 2))) + + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + + onExecutorAddedDefaultProfile(manager, "0") + val t1 = createTaskInfo(0, 0, executorId = s"0") + val t2 = createTaskInfo(1, 1, executorId = s"0") + post(SparkListenerTaskStart(0, 0, t1)) + post(SparkListenerTaskStart(0, 0, t2)) + + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + + // Stage 0 becomes unschedulable due to blacklisting + post(SparkListenerUnschedulableTaskSetAdded(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + // Assert that we are getting additional executor to schedule unschedulable tasks + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) + + // Add a new executor + onExecutorAddedDefaultProfile(manager, "1") + // Now once the task becomes schedulable, clear the unschedulableTaskSets + post(SparkListenerUnschedulableTaskSetRemoved(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + } + + test("SPARK-31418: multiple stages being unschedulable") { + val clock = new ManualClock() + val conf = createConf(0, 10, 0).set(config.EXECUTOR_CORES, 2) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(createStageInfo(0, 2))) + post(SparkListenerStageSubmitted(createStageInfo(1, 2))) + post(SparkListenerStageSubmitted(createStageInfo(2, 2))) + + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + + // Add necessary executors + (0 to 2).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) + + // Start all the tasks + (0 to 2).foreach { + i => + val t1Info = createTaskInfo(0, (i * 2) + 1, executorId = s"${i / 2}") + val t2Info = createTaskInfo(1, (i * 2) + 2, executorId = s"${i / 2}") + post(SparkListenerTaskStart(i, 0, t1Info)) + post(SparkListenerTaskStart(i, 0, t2Info)) + } + assert(numExecutorsTarget(manager, defaultProfile.id) === 3) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 3) + + // Complete the stage 0 tasks. + val t1Info = createTaskInfo(0, 0, executorId = s"0") + val t2Info = createTaskInfo(1, 1, executorId = s"0") + post(SparkListenerTaskEnd(0, 0, null, Success, t1Info, new ExecutorMetrics, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, t2Info, new ExecutorMetrics, null)) + post(SparkListenerStageCompleted(createStageInfo(0, 2))) + + // Stage 1 and 2 becomes unschedulable now due to blacklisting + post(SparkListenerUnschedulableTaskSetAdded(1, 0)) + post(SparkListenerUnschedulableTaskSetAdded(2, 0)) + + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + // Assert that we are getting additional executor to schedule unschedulable tasks + assert(numExecutorsTarget(manager, defaultProfile.id) === 4) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 4) + + // Add a new executor + onExecutorAddedDefaultProfile(manager, "3") + + // Now once the task becomes schedulable, clear the unschedulableTaskSets + post(SparkListenerUnschedulableTaskSetRemoved(1, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 4) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + } + + test("SPARK-31418: remove executors after unschedulable tasks end") { + val clock = new ManualClock() + val stage = createStageInfo(0, 10) + val conf = createConf(0, 6, 0).set(config.EXECUTOR_CORES, 2) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(stage)) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + + (0 to 4).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) + (0 to 9).map { i => createTaskInfo(i, i, executorId = s"${i / 2}") }.foreach { + info => post(SparkListenerTaskStart(0, 0, info)) + } + assert(numExecutorsTarget(manager, defaultProfile.id) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + + // 8 tasks (0 - 7) finished + (0 to 7).map { i => createTaskInfo(i, i, executorId = s"${i / 2}") }.foreach { + info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) + } + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + (0 to 3).foreach { i => assert(removeExecutorDefaultProfile(manager, i.toString)) } + (0 to 3).foreach { i => onExecutorRemoved(manager, i.toString) } + + // Now due to blacklisting, the task becomes unschedulable + post(SparkListenerUnschedulableTaskSetAdded(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) + + // New executor got added + onExecutorAddedDefaultProfile(manager, "5") + + // Now once the task becomes schedulable, clear the unschedulableTaskSets + post(SparkListenerUnschedulableTaskSetRemoved(0, 0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(9, 9, "4"), new ExecutorMetrics, null)) + // Unschedulable task successfully ran on the new executor provisioned + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(8, 8, "5"), new ExecutorMetrics, null)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + post(SparkListenerStageCompleted(stage)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager, defaultProfile.id) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 0) + assert(removeExecutorDefaultProfile(manager, "4")) + onExecutorRemoved(manager, "4") + assert(removeExecutorDefaultProfile(manager, "5")) + onExecutorRemoved(manager, "5") + } + test("SPARK-30511 remove executors when speculative tasks end") { val clock = new ManualClock() val stage = createStageInfo(0, 40) @@ -1434,7 +1603,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { .set(config.DYN_ALLOCATION_TESTING, true) // SPARK-22864: effectively disable the allocation schedule by setting the period to a // really long value. - .set(TEST_SCHEDULE_INTERVAL, 10000L) + .set(TEST_SCHEDULE_INTERVAL, 30000L) sparkConf } diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 65391db405a55..9026447e5a98c 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.apache.spark.internal.config diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala index 70b6309be7d53..737a856ba13f1 100644 --- a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark import scala.concurrent.duration.Duration -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.util.ThreadUtils diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 94ad8d8880027..082a92ef41d3b 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -25,7 +25,7 @@ import scala.concurrent.Future import scala.concurrent.duration._ import org.scalatest.BeforeAndAfter -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Deploy._ diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 3d6690cb85348..5ea43e2dac7d3 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark import java.util.{Locale, Properties} import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.internal.config diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index 1a563621a5179..3d853ff4294be 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.internal.config import org.apache.spark.rdd.ShuffledRDD diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 3bc2061c4f2ad..72e7ee0214187 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -221,7 +221,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst conf.registerKryoClasses(Array(classOf[Class1])) assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === Seq(classOf[Class1].getName).toSet) - conf.set(KRYO_USER_REGISTRATORS, classOf[CustomRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[CustomRegistrator].getName)) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index d111bb33ce8ff..ebdf2f59a2770 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -31,8 +31,8 @@ import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.json4s.{DefaultFormats, Extraction} -import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ @@ -946,6 +946,15 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(error.contains("SparkContext should only be created and accessed on the driver.")) } + + test("SPARK-32160: Allow to create SparkContext in executors if the config is set") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[3, 1, 1024]")) + + sc.range(0, 1).foreach { _ => + new SparkContext(new SparkConf().setAppName("test").setMaster("local") + .set(EXECUTOR_ALLOW_SPARK_CONTEXT, true)).stop() + } + } } object SparkContextSuite { diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index ec641f8294b29..581786ce0a70c 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -25,7 +25,8 @@ import org.apache.log4j.spi.LoggingEvent import scala.annotation.tailrec import org.apache.log4j.{Appender, AppenderSkeleton, Level, Logger} -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, Outcome} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, Outcome} +import org.scalatest.funsuite.AnyFunSuite import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.util.{AccumulatorContext, Utils} @@ -57,7 +58,7 @@ import scala.collection.mutable.ArrayBuffer * } */ abstract class SparkFunSuite - extends FunSuite + extends AnyFunSuite with BeforeAndAfterAll with BeforeAndAfterEach with ThreadAudit diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index f527bbe718524..fae6c4af1240c 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -20,8 +20,9 @@ package org.apache.spark import scala.concurrent.duration._ import scala.language.implicitConversions -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.JobExecutionStatus._ diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala index dffdd96cd2dcc..daf0151ad65a7 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -21,7 +21,8 @@ import java.io.{File, PrintWriter} import scala.io.Source -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} import org.apache.spark.internal.config.Kryo._ diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 6a99dbca64f4b..792168834dea2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.deploy -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala new file mode 100644 index 0000000000000..ee9a6be03868f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala @@ -0,0 +1,424 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.duration._ + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark._ +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState, WorkerDecommission} +import org.apache.spark.deploy.master.{ApplicationInfo, Master, WorkerInfo} +import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.network.TransportContext +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.ExternalBlockHandler +import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.scheduler._ +import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.Utils + +class DecommissionWorkerSuite + extends SparkFunSuite + with Logging + with LocalSparkContext + with BeforeAndAfterEach { + + private var masterAndWorkerConf: SparkConf = null + private var masterAndWorkerSecurityManager: SecurityManager = null + private var masterRpcEnv: RpcEnv = null + private var master: Master = null + private var workerIdToRpcEnvs: mutable.HashMap[String, RpcEnv] = null + private var workers: mutable.ArrayBuffer[Worker] = null + + override def beforeEach(): Unit = { + super.beforeEach() + masterAndWorkerConf = new SparkConf() + .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) + masterAndWorkerSecurityManager = new SecurityManager(masterAndWorkerConf) + masterRpcEnv = RpcEnv.create( + Master.SYSTEM_NAME, + "localhost", + 0, + masterAndWorkerConf, + masterAndWorkerSecurityManager) + master = makeMaster() + workerIdToRpcEnvs = mutable.HashMap.empty + workers = mutable.ArrayBuffer.empty + } + + override def afterEach(): Unit = { + try { + masterRpcEnv.shutdown() + workerIdToRpcEnvs.values.foreach(_.shutdown()) + workerIdToRpcEnvs.clear() + master.stop() + workers.foreach(_.stop()) + workers.clear() + masterRpcEnv = null + } finally { + super.afterEach() + } + } + + test("decommission workers should not result in job failure") { + val maxTaskFailures = 2 + val numTimesToKillWorkers = maxTaskFailures + 1 + val numWorkers = numTimesToKillWorkers + 1 + createWorkers(numWorkers) + + // Here we will have a single task job and we will keep decommissioning (and killing) the + // worker running that task K times. Where K is more than the maxTaskFailures. Since the worker + // is notified of the decommissioning, the task failures can be ignored and not fail + // the job. + + sc = createSparkContext(config.TASK_MAX_FAILURES.key -> maxTaskFailures.toString) + val executorIdToWorkerInfo = getExecutorToWorkerAssignments + val taskIdsKilled = new ConcurrentHashMap[Long, Boolean] + val listener = new RootStageAwareListener { + override def handleRootTaskStart(taskStart: SparkListenerTaskStart): Unit = { + val taskInfo = taskStart.taskInfo + if (taskIdsKilled.size() < numTimesToKillWorkers) { + val workerInfo = executorIdToWorkerInfo(taskInfo.executorId) + decommissionWorkerOnMaster(workerInfo, "partition 0 must die") + killWorkerAfterTimeout(workerInfo, 1) + taskIdsKilled.put(taskInfo.taskId, true) + } + } + } + TestUtils.withListener(sc, listener) { _ => + val jobResult = sc.parallelize(1 to 1, 1).map { _ => + Thread.sleep(5 * 1000L); 1 + }.count() + assert(jobResult === 1) + } + // single task job that gets to run numTimesToKillWorkers + 1 times. + assert(listener.getTasksFinished().size === numTimesToKillWorkers + 1) + listener.rootTasksStarted.asScala.foreach { taskInfo => + assert(taskInfo.index == 0, s"Unknown task index ${taskInfo.index}") + } + listener.rootTasksEnded.asScala.foreach { taskInfo => + assert(taskInfo.index === 0, s"Expected task index ${taskInfo.index} to be 0") + // If a task has been killed then it shouldn't be successful + val taskSuccessExpected = !taskIdsKilled.getOrDefault(taskInfo.taskId, false) + val taskSuccessActual = taskInfo.successful + assert(taskSuccessActual === taskSuccessExpected, + s"Expected task success $taskSuccessActual == $taskSuccessExpected") + } + } + + test("decommission workers ensure that shuffle output is regenerated even with shuffle service") { + createWorkers(2) + val ss = new ExternalShuffleServiceHolder() + + sc = createSparkContext( + config.Tests.TEST_NO_STAGE_RETRY.key -> "true", + config.SHUFFLE_MANAGER.key -> "sort", + config.SHUFFLE_SERVICE_ENABLED.key -> "true", + config.SHUFFLE_SERVICE_PORT.key -> ss.getPort.toString + ) + + // Here we will create a 2 stage job: The first stage will have two tasks and the second stage + // will have one task. The two tasks in the first stage will be long and short. We decommission + // and kill the worker after the short task is done. Eventually the driver should get the + // executor lost signal for the short task executor. This should trigger regenerating + // the shuffle output since we cleanly decommissioned the executor, despite running with an + // external shuffle service. + try { + val executorIdToWorkerInfo = getExecutorToWorkerAssignments + val workerForTask0Decommissioned = new AtomicBoolean(false) + // single task job + val listener = new RootStageAwareListener { + override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskInfo = taskEnd.taskInfo + if (taskInfo.index == 0) { + if (workerForTask0Decommissioned.compareAndSet(false, true)) { + val workerInfo = executorIdToWorkerInfo(taskInfo.executorId) + decommissionWorkerOnMaster(workerInfo, "Kill early done map worker") + killWorkerAfterTimeout(workerInfo, 0) + logInfo(s"Killed the node ${workerInfo.hostPort} that was running the early task") + } + } + } + } + TestUtils.withListener(sc, listener) { _ => + val jobResult = sc.parallelize(1 to 2, 2).mapPartitionsWithIndex((pid, _) => { + val sleepTimeSeconds = if (pid == 0) 1 else 10 + Thread.sleep(sleepTimeSeconds * 1000L) + List(1).iterator + }, preservesPartitioning = true).repartition(1).sum() + assert(jobResult === 2) + } + val tasksSeen = listener.getTasksFinished() + // 4 tasks: 2 from first stage, one retry due to decom, one more from the second stage. + assert(tasksSeen.size === 4, s"Expected 4 tasks but got $tasksSeen") + listener.rootTasksStarted.asScala.foreach { taskInfo => + assert(taskInfo.index <= 1, s"Expected ${taskInfo.index} <= 1") + assert(taskInfo.successful, s"Task ${taskInfo.index} should be successful") + } + val tasksEnded = listener.rootTasksEnded.asScala + tasksEnded.filter(_.index != 0).foreach { taskInfo => + assert(taskInfo.attemptNumber === 0, "2nd task should succeed on 1st attempt") + } + val firstTaskAttempts = tasksEnded.filter(_.index == 0) + assert(firstTaskAttempts.size > 1, s"Task 0 should have multiple attempts") + } finally { + ss.close() + } + } + + test("decommission workers ensure that fetch failures lead to rerun") { + createWorkers(2) + sc = createSparkContext( + config.Tests.TEST_NO_STAGE_RETRY.key -> "false", + config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE.key -> "true") + val executorIdToWorkerInfo = getExecutorToWorkerAssignments + val executorToDecom = executorIdToWorkerInfo.keysIterator.next + + // The task code below cannot call executorIdToWorkerInfo, so we need to pre-compute + // the worker to decom to force it to be serialized into the task. + val workerToDecom = executorIdToWorkerInfo(executorToDecom) + + // The setup of this job is similar to the one above: 2 stage job with first stage having + // long and short tasks. Except that we want the shuffle output to be regenerated on a + // fetch failure instead of an executor lost. Since it is hard to "trigger a fetch failure", + // we manually raise the FetchFailed exception when the 2nd stage's task runs and require that + // fetch failure to trigger a recomputation. + logInfo(s"Will try to decommission the task running on executor $executorToDecom") + val listener = new RootStageAwareListener { + override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskInfo = taskEnd.taskInfo + if (taskInfo.executorId == executorToDecom && taskInfo.attemptNumber == 0 && + taskEnd.stageAttemptId == 0) { + decommissionWorkerOnMaster(workerToDecom, + "decommission worker after task on it is done") + } + } + } + TestUtils.withListener(sc, listener) { _ => + val jobResult = sc.parallelize(1 to 2, 2).mapPartitionsWithIndex((_, _) => { + val executorId = SparkEnv.get.executorId + val sleepTimeSeconds = if (executorId == executorToDecom) 10 else 1 + Thread.sleep(sleepTimeSeconds * 1000L) + List(1).iterator + }, preservesPartitioning = true) + .repartition(1).mapPartitions(iter => { + val context = TaskContext.get() + if (context.attemptNumber == 0 && context.stageAttemptNumber() == 0) { + // MapIndex is explicitly -1 to force the entire host to be decommissioned + // However, this will cause both the tasks in the preceding stage since the host here is + // "localhost" (shortcoming of this single-machine unit test in that all the workers + // are actually on the same host) + throw new FetchFailedException(BlockManagerId(executorToDecom, + workerToDecom.host, workerToDecom.port), 0, 0, -1, 0, "Forcing fetch failure") + } + val sumVal: List[Int] = List(iter.sum) + sumVal.iterator + }, preservesPartitioning = true) + .sum() + assert(jobResult === 2) + } + // 6 tasks: 2 from first stage, 2 rerun again from first stage, 2nd stage attempt 1 and 2. + val tasksSeen = listener.getTasksFinished() + assert(tasksSeen.size === 6, s"Expected 6 tasks but got $tasksSeen") + } + + private abstract class RootStageAwareListener extends SparkListener { + private var rootStageId: Option[Int] = None + private val tasksFinished = new ConcurrentLinkedQueue[String]() + private val jobDone = new AtomicBoolean(false) + val rootTasksStarted = new ConcurrentLinkedQueue[TaskInfo]() + val rootTasksEnded = new ConcurrentLinkedQueue[TaskInfo]() + + protected def isRootStageId(stageId: Int): Boolean = + (rootStageId.isDefined && rootStageId.get == stageId) + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + if (stageSubmitted.stageInfo.parentIds.isEmpty && rootStageId.isEmpty) { + rootStageId = Some(stageSubmitted.stageInfo.stageId) + } + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + jobEnd.jobResult match { + case JobSucceeded => jobDone.set(true) + } + } + + protected def handleRootTaskEnd(end: SparkListenerTaskEnd) = {} + + protected def handleRootTaskStart(start: SparkListenerTaskStart) = {} + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + if (isRootStageId(taskStart.stageId)) { + rootTasksStarted.add(taskStart.taskInfo) + handleRootTaskStart(taskStart) + } + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskSignature = s"${taskEnd.stageId}:${taskEnd.stageAttemptId}:" + + s"${taskEnd.taskInfo.index}:${taskEnd.taskInfo.attemptNumber}" + logInfo(s"Task End $taskSignature") + tasksFinished.add(taskSignature) + if (isRootStageId(taskEnd.stageId)) { + rootTasksEnded.add(taskEnd.taskInfo) + handleRootTaskEnd(taskEnd) + } + } + + def getTasksFinished(): Seq[String] = { + assert(jobDone.get(), "Job isn't successfully done yet") + tasksFinished.asScala.toSeq + } + } + + private def getExecutorToWorkerAssignments: Map[String, WorkerInfo] = { + val executorIdToWorkerInfo = mutable.HashMap[String, WorkerInfo]() + master.workers.foreach { wi => + assert(wi.executors.size <= 1, "There should be at most one executor per worker") + // Cast the executorId to string since the TaskInfo.executorId is a string + wi.executors.values.foreach { e => + val executorIdString = e.id.toString + val oldWorkerInfo = executorIdToWorkerInfo.put(executorIdString, wi) + assert(oldWorkerInfo.isEmpty, + s"Executor $executorIdString already present on another worker ${oldWorkerInfo}") + } + } + executorIdToWorkerInfo.toMap + } + + private def makeMaster(): Master = { + val master = new Master( + masterRpcEnv, + masterRpcEnv.address, + 0, + masterAndWorkerSecurityManager, + masterAndWorkerConf) + masterRpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + master + } + + private def createWorkers(numWorkers: Int, cores: Int = 1, memory: Int = 1024): Unit = { + val workerRpcEnvs = (0 until numWorkers).map { i => + RpcEnv.create( + Worker.SYSTEM_NAME + i, + "localhost", + 0, + masterAndWorkerConf, + masterAndWorkerSecurityManager) + } + workers.clear() + val rpcAddressToRpcEnv: mutable.HashMap[RpcAddress, RpcEnv] = mutable.HashMap.empty + workerRpcEnvs.foreach { rpcEnv => + val workDir = Utils.createTempDir(namePrefix = this.getClass.getSimpleName()).toString + val worker = new Worker(rpcEnv, 0, cores, memory, Array(masterRpcEnv.address), + Worker.ENDPOINT_NAME, workDir, masterAndWorkerConf, masterAndWorkerSecurityManager) + rpcEnv.setupEndpoint(Worker.ENDPOINT_NAME, worker) + workers.append(worker) + val oldRpcEnv = rpcAddressToRpcEnv.put(rpcEnv.address, rpcEnv) + logInfo(s"Created a worker at ${rpcEnv.address} with workdir $workDir") + assert(oldRpcEnv.isEmpty, s"Detected duplicate rpcEnv ${oldRpcEnv} for ${rpcEnv.address}") + } + workerIdToRpcEnvs.clear() + // Wait until all workers register with master successfully + eventually(timeout(1.minute), interval(1.seconds)) { + val workersOnMaster = getMasterState.workers + val numWorkersCurrently = workersOnMaster.length + logInfo(s"Waiting for $numWorkers workers to come up: So far $numWorkersCurrently") + assert(numWorkersCurrently === numWorkers) + workersOnMaster.foreach { workerInfo => + val rpcAddress = RpcAddress(workerInfo.host, workerInfo.port) + val rpcEnv = rpcAddressToRpcEnv(rpcAddress) + assert(rpcEnv != null, s"Cannot find the worker for $rpcAddress") + val oldRpcEnv = workerIdToRpcEnvs.put(workerInfo.id, rpcEnv) + assert(oldRpcEnv.isEmpty, s"Detected duplicate rpcEnv ${oldRpcEnv} for worker " + + s"${workerInfo.id}") + } + } + logInfo(s"Created ${workers.size} workers") + } + + private def getMasterState: MasterStateResponse = { + master.self.askSync[MasterStateResponse](RequestMasterState) + } + + private def getApplications(): Seq[ApplicationInfo] = { + getMasterState.activeApps + } + + def decommissionWorkerOnMaster(workerInfo: WorkerInfo, reason: String): Unit = { + logInfo(s"Trying to decommission worker ${workerInfo.id} for reason `$reason`") + master.self.send(WorkerDecommission(workerInfo.id, workerInfo.endpoint)) + } + + def killWorkerAfterTimeout(workerInfo: WorkerInfo, secondsToWait: Int): Unit = { + val env = workerIdToRpcEnvs(workerInfo.id) + Thread.sleep(secondsToWait * 1000L) + env.shutdown() + env.awaitTermination() + } + + def createSparkContext(extraConfs: (String, String)*): SparkContext = { + val conf = new SparkConf() + .setMaster(masterRpcEnv.address.toSparkURL) + .setAppName("test") + .setAll(extraConfs) + sc = new SparkContext(conf) + val appId = sc.applicationId + eventually(timeout(1.minute), interval(1.seconds)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.getExecutorLimit === Int.MaxValue) + } + sc + } + + private class ExternalShuffleServiceHolder() { + // The external shuffle service can start with default configs and not get polluted by the + // other configs used in this test. + private val transportConf = SparkTransportConf.fromSparkConf(new SparkConf(), + "shuffle", numUsableCores = 2) + private val rpcHandler = new ExternalBlockHandler(transportConf, null) + private val transportContext = new TransportContext(transportConf, rpcHandler) + private val server = transportContext.createServer() + + def getPort: Int = server.getPort + + def close(): Unit = { + Utils.tryLogNonFatalError { + server.close() + } + Utils.tryLogNonFatalError { + rpcHandler.close() + } + Utils.tryLogNonFatalError { + transportContext.close() + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 354e6eb2138d9..7d3eb7c6b0f6e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -128,8 +128,8 @@ object JsonConstants { |:["3","4","5"]}},"resourcesused":{"gpu": |{"name":"gpu","addresses":[]},"fpga": |{"name":"fpga","addresses":[]}},"resourcesfree": - |{"gpu":{"name":"gpu","addresses":["2","1","0"]}, - |"fpga":{"name":"fpga","addresses":["5","4","3"]}}, + |{"gpu":{"name":"gpu","addresses":["0","1","2"]}, + |"fpga":{"name":"fpga","addresses":["3","4","5"]}}, |"state":"ALIVE","lastheartbeat":%d} """.format(currTimeInMillis).stripMargin 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 fd3d4bcf62f69..35311d372e478 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -29,8 +29,10 @@ import com.google.common.io.ByteStreams import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FSDataInputStream, Path} -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index a3e39d7f53728..e091bd05c2dc8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.client import java.io.Closeable -import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} import scala.concurrent.duration._ @@ -32,6 +32,7 @@ import org.apache.spark.deploy.master.{ApplicationInfo, Master} import org.apache.spark.deploy.worker.Worker import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv +import org.apache.spark.scheduler.ExecutorDecommissionInfo import org.apache.spark.util.Utils /** @@ -126,7 +127,10 @@ class AppClientSuite // Decommissioning is async. eventually(timeout(1.seconds), interval(10.millis)) { // We only record decommissioning for the executor we've requested - assert(ci.listener.execDecommissionedList.size === 1) + assert(ci.listener.execDecommissionedMap.size === 1) + val decommissionInfo = ci.listener.execDecommissionedMap.get(executorId) + assert(decommissionInfo != null && decommissionInfo.isHostDecommissioned, + s"$executorId should have been decommissioned along with its worker") } // Send request to kill executor, verify request was made @@ -215,7 +219,7 @@ class AppClientSuite val deadReasonList = new ConcurrentLinkedQueue[String]() val execAddedList = new ConcurrentLinkedQueue[String]() val execRemovedList = new ConcurrentLinkedQueue[String]() - val execDecommissionedList = new ConcurrentLinkedQueue[String]() + val execDecommissionedMap = new ConcurrentHashMap[String, ExecutorDecommissionInfo]() def connected(id: String): Unit = { connectedIdList.add(id) @@ -245,8 +249,9 @@ class AppClientSuite execRemovedList.add(id) } - def executorDecommissioned(id: String, message: String): Unit = { - execDecommissionedList.add(id) + def executorDecommissioned(id: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { + val previousDecommissionInfo = execDecommissionedMap.putIfAbsent(id, decommissionInfo) + assert(previousDecommissionInfo === null, s"Expected no previous decommission info for $id") } def workerRemoved(workerId: String, host: String, message: String): Unit = {} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala index 48bd088d07ff9..566eb9cf3c143 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala @@ -27,7 +27,8 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index ade03a0095c19..21a99a462aa1e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -34,8 +34,9 @@ import org.apache.hadoop.security.AccessControlException import org.json4s.jackson.JsonMethods._ import org.mockito.ArgumentMatchers.{any, argThat} import org.mockito.Mockito.{doThrow, mock, spy, verify, when} -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{JobExecutionStatus, SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 39b339caea385..51e38f9cdcd2d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -35,8 +35,10 @@ import org.json4s.jackson.JsonMethods._ import org.mockito.Mockito._ import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.scalatestplus.selenium.WebBrowser @@ -176,6 +178,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor memory usage" -> "applications/app-20161116163331-0000/executors", "executor resource information" -> "applications/application_1555004656427_0144/executors", "multiple resource profiles" -> "applications/application_1578436911597_0052/environment", + "stage list with peak metrics" -> "applications/app-20200706201101-0003/stages", + "stage with peak metrics" -> "applications/app-20200706201101-0003/stages/2/0", "app environment" -> "applications/app-20161116163331-0000/environment", @@ -309,14 +313,18 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val urlsThroughKnox = responseThroughKnox \\ "@href" map (_.toString) val siteRelativeLinksThroughKnox = urlsThroughKnox filter (_.startsWith("/")) - all (siteRelativeLinksThroughKnox) should startWith (knoxBaseUrl) + for (link <- siteRelativeLinksThroughKnox) { + link should startWith (knoxBaseUrl) + } val directRequest = mock[HttpServletRequest] val directResponse = page.render(directRequest) val directUrls = directResponse \\ "@href" map (_.toString) val directSiteRelativeLinks = directUrls filter (_.startsWith("/")) - all (directSiteRelativeLinks) should not startWith (knoxBaseUrl) + for (link <- directSiteRelativeLinks) { + link should not startWith (knoxBaseUrl) + } } test("static relative links are prefixed with uiRoot (spark.ui.proxyBase)") { @@ -331,7 +339,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // then val urls = response \\ "@href" map (_.toString) val siteRelativeLinks = urls filter (_.startsWith("/")) - all (siteRelativeLinks) should startWith (uiRoot) + for (link <- siteRelativeLinks) { + link should startWith (uiRoot) + } } test("/version api endpoint") { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala index 8a1e22c694497..4a5c34f86753c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/RealBrowserUIHistoryServerSuite.scala @@ -22,7 +22,9 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.proxy.ProxyServlet import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.openqa.selenium.WebDriver -import org.scalatest._ +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.selenium.WebBrowser import org.apache.spark._ @@ -146,7 +148,9 @@ abstract class RealBrowserUIHistoryServerSuite(val driverProp: String) // there are at least some URL links that were generated via javascript, // and they all contain the spark.ui.proxyBase (uiRoot) links.length should be > 4 - all(links) should startWith(url + uiRoot) + for (link <- links) { + link should startWith(url + uiRoot) + } } finally { contextHandler.stop() quit() diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 91128af82b022..8898d68664f36 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.duration._ import scala.io.Source import scala.reflect.ClassTag @@ -31,8 +31,10 @@ import scala.reflect.ClassTag import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, Matchers, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} @@ -726,6 +728,65 @@ class MasterSuite extends SparkFunSuite } } + def testWorkerDecommissioning( + numWorkers: Int, + numWorkersExpectedToDecom: Int, + hostnames: Seq[String]): Unit = { + val conf = new SparkConf() + val master = makeAliveMaster(conf) + val workerRegs = (1 to numWorkers).map{idx => + val worker = new MockWorker(master.self, conf) + worker.rpcEnv.setupEndpoint("worker", worker) + val workerReg = RegisterWorker( + worker.id, + "localhost", + worker.self.address.port, + worker.self, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost", 10000)) + master.self.send(workerReg) + workerReg + } + + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers.length === numWorkers) + assert(masterState.workers.forall(_.state == WorkerState.ALIVE)) + assert(masterState.workers.map(_.id).toSet == workerRegs.map(_.id).toSet) + } + + val decomWorkersCount = master.self.askSync[Integer](DecommissionWorkersOnHosts(hostnames)) + assert(decomWorkersCount === numWorkersExpectedToDecom) + + // Decommissioning is actually async ... wait for the workers to actually be decommissioned by + // polling the master's state. + eventually(timeout(30.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers.length === numWorkers) + val workersActuallyDecomed = masterState.workers.count(_.state == WorkerState.DECOMMISSIONED) + assert(workersActuallyDecomed === numWorkersExpectedToDecom) + } + + // Decommissioning a worker again should return the same answer since we want this call to be + // idempotent. + val decomWorkersCountAgain = master.self.askSync[Integer](DecommissionWorkersOnHosts(hostnames)) + assert(decomWorkersCountAgain === numWorkersExpectedToDecom) + } + + test("All workers on a host should be decommissioned") { + testWorkerDecommissioning(2, 2, Seq("LoCalHost", "localHOST")) + } + + test("No workers should be decommissioned with invalid host") { + testWorkerDecommissioning(2, 0, Seq("NoSuchHost1", "NoSuchHost2")) + } + + test("Only worker on host should be decommissioned") { + testWorkerDecommissioning(1, 1, Seq("lOcalHost", "NoSuchHost")) + } + test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { val conf = new SparkConf().set(WORKER_TIMEOUT, 1L) val master = makeAliveMaster(conf) diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index e2d7facdd77e0..35de457ec48ce 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -21,6 +21,7 @@ import java.io.DataOutputStream import java.net.{HttpURLConnection, URL} import java.nio.charset.StandardCharsets import java.util.Date +import javax.servlet.http.HttpServletResponse import scala.collection.mutable.HashMap @@ -28,15 +29,16 @@ import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, RequestKillDriver} +import org.apache.spark.deploy.DeployMessages.{DecommissionWorkersOnHosts, KillDriverResponse, RequestKillDriver} import org.apache.spark.deploy.DeployTestUtils._ import org.apache.spark.deploy.master._ +import org.apache.spark.internal.config.UI import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { - val conf = new SparkConf + val conf = new SparkConf() val securityMgr = new SecurityManager(conf) val rpcEnv = mock(classOf[RpcEnv]) val master = mock(classOf[Master]) @@ -88,10 +90,32 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { verify(masterEndpointRef, times(1)).ask[KillDriverResponse](RequestKillDriver(activeDriverId)) } - private def convPostDataToString(data: Map[String, String]): String = { + private def testKillWorkers(hostnames: Seq[String]): Unit = { + val url = s"http://localhost:${masterWebUI.boundPort}/workers/kill/" + val body = convPostDataToString(hostnames.map(("host", _))) + val conn = sendHttpRequest(url, "POST", body) + // The master is mocked here, so cannot assert on the response code + conn.getResponseCode + // Verify that master was asked to kill driver with the correct id + verify(masterEndpointRef).askSync[Integer](DecommissionWorkersOnHosts(hostnames)) + } + + test("Kill one host") { + testKillWorkers(Seq("localhost")) + } + + test("Kill multiple hosts") { + testKillWorkers(Seq("noSuchHost", "LocalHost")) + } + + private def convPostDataToString(data: Seq[(String, String)]): String = { (for ((name, value) <- data) yield s"$name=$value").mkString("&") } + private def convPostDataToString(data: Map[String, String]): String = { + convPostDataToString(data.toSeq) + } + /** * Send an HTTP request to the given URL using the method and the body specified. * Return the connection object. diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 03102fd8c696c..d08052faa0043 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.rest import java.lang.Boolean +import scala.util.Properties.versionNumberString + import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkFunSuite} @@ -232,7 +234,7 @@ class SubmitRestProtocolSuite extends SparkFunSuite { |} """.stripMargin - private val submitDriverRequestJson = + private lazy val submitDriverRequestJson = if (versionNumberString.startsWith("2.12")) { s""" |{ | "action" : "CreateSubmissionRequest", @@ -258,6 +260,33 @@ class SubmitRestProtocolSuite extends SparkFunSuite { | } |} """.stripMargin + } else { + s""" + |{ + | "action" : "CreateSubmissionRequest", + | "appArgs" : [ "two slices", "a hint of cinnamon" ], + | "appResource" : "honey-walnut-cherry.jar", + | "clientSparkVersion" : "1.2.3", + | "environmentVariables" : { + | "PATH" : "/dev/null" + | }, + | "mainClass" : "org.apache.spark.examples.SparkPie", + | "sparkProperties" : { + | "spark.driver.extraLibraryPath" : "pickle.jar", + | "spark.jars" : "mayonnaise.jar,ketchup.jar", + | "spark.driver.supervise" : "false", + | "spark.driver.memory" : "${Utils.DEFAULT_DRIVER_MEM_MB}m", + | "spark.files" : "fireball.png", + | "spark.driver.cores" : "180", + | "spark.driver.extraJavaOptions" : " -Dslices=5 -Dcolor=mostly_red", + | "spark.app.name" : "SparkPie", + | "spark.cores.max" : "10000", + | "spark.executor.memory" : "256m", + | "spark.driver.extraClassPath" : "food-coloring.jar" + | } + |} + """.stripMargin + } private val submitDriverResponseJson = """ diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala index 44f38e7043dcd..2cfc60f927e1c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config.{STAGING_DIR, SUBMIT_DEPLOY_MODE} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala index 2d3cc5d3abd65..28e35bc8183ba 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.deploy.worker -import org.scalatest.{Matchers, PrivateMethodTester} +import org.scalatest.PrivateMethodTester +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.Command diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index 2d3d0afe3f80c..5bbd60f99f77e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -29,8 +29,10 @@ import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.TestUtils.{createTempJsonFile, createTempScriptWithExpectedOutput} diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 31049d104e63d..8e58beff74290 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.io.{Externalizable, File, ObjectInput, ObjectOutput} import java.lang.Thread.UncaughtExceptionHandler import java.nio.ByteBuffer import java.util.Properties @@ -41,6 +41,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.broadcast.Broadcast +import org.apache.spark.deploy.{SimpleApplicationTest, SparkSubmitSuite} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.memory.TestMemoryManager @@ -52,7 +53,7 @@ import org.apache.spark.scheduler.{DirectTaskResult, FakeTask, ResultTask, Task, import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockManager, BlockManagerId} -import org.apache.spark.util.{LongAccumulator, UninterruptibleThread} +import org.apache.spark.util.{LongAccumulator, UninterruptibleThread, Utils} class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with Eventually with PrivateMethodTester { diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala index edec968d0745a..473782ee28d1c 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -21,8 +21,9 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration._ -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.internal.config.UI.UI_ENABLED diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index c726329ce8a84..13bb811b840d5 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -28,7 +28,8 @@ import scala.util.{Failure, Success, Try} import com.google.common.io.CharStreams import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala index c804102e4ab2c..baa878eb14047 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala @@ -25,7 +25,9 @@ import scala.util.Random import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, times, verify, when} -import org.scalatest._ +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{ExecutorDeadException, SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.network.BlockDataManager diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 05013fbc49b8e..a204502be74b6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -26,7 +26,8 @@ import org.apache.spark.util.Utils class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { - before { + override def beforeAll(): Unit = { + super.beforeAll() Utils.classForName("org.apache.derby.jdbc.EmbeddedDriver") val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true") try { @@ -96,7 +97,7 @@ class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkCont assert(rdd.reduce(_ + _) === 5050) } - after { + override def afterAll(): Unit = { try { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;shutdown=true") } catch { @@ -104,5 +105,6 @@ class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkCont // Normal single database shutdown // https://db.apache.org/derby/docs/10.2/ref/rrefexcept71493.html } + super.afterAll() } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 2da2854dfbcb9..5000011b3c5ee 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -176,7 +176,8 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext with Eventuall } test("pipe with env variable") { - assume(TestUtils.testCommandAvailable(envCommand)) + val executable = envCommand.split("\\s+", 2)(0) + assume(TestUtils.testCommandAvailable(executable)) val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe(s"$envCommand MY_TEST_ENV", Map("MY_TEST_ENV" -> "LALALA")) val c = piped.collect() @@ -238,7 +239,8 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext with Eventuall } def testExportInputFile(varName: String): Unit = { - assume(TestUtils.testCommandAvailable(envCommand)) + val executable = envCommand.split("\\s+", 2)(0) + assume(TestUtils.testCommandAvailable(executable)) val nums = new HadoopRDD(sc, new JobConf(), classOf[TextInputFormat], classOf[LongWritable], classOf[Text], 2) { override def getPartitions: Array[Partition] = Array(generateFakeHadoopPartition()) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 79f9c1396c87b..8962fd6740bf6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -237,10 +237,8 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { test("aggregate") { val pairs = sc.makeRDD(Seq(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) - type StringMap = HashMap[String, Int] - val emptyMap = new StringMap { - override def default(key: String): Int = 0 - } + type StringMap = scala.collection.mutable.Map[String, Int] + val emptyMap = HashMap[String, Int]().withDefaultValue(0).asInstanceOf[StringMap] val mergeElement: (StringMap, (String, Int)) => StringMap = (map, pair) => { map(pair._1) += pair._2 map diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index d5f7d30a253fe..5b01b54a0a9f4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.rdd -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SharedSparkContext, SparkFunSuite} import org.apache.spark.internal.Logging diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index 29d3ef130ce57..d0479ca7db40c 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.resource import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY import org.apache.spark.resource.TestResourceIDs._ @@ -55,6 +55,8 @@ class ResourceProfileSuite extends SparkFunSuite { "pyspark memory empty if not specified") assert(rprof.executorResources.get(ResourceProfile.OVERHEAD_MEM) == None, "overhead memory empty if not specified") + assert(rprof.executorResources.get(ResourceProfile.OFFHEAP_MEM) == None, + "offHeap memory empty if not specified") assert(rprof.taskResources.size === 1, "Task resources should just contain cpus by default") assert(rprof.taskResources(ResourceProfile.CPUS).amount === 1, @@ -69,14 +71,16 @@ class ResourceProfileSuite extends SparkFunSuite { conf.set(EXECUTOR_MEMORY_OVERHEAD.key, "1g") conf.set(EXECUTOR_MEMORY.key, "4g") conf.set(EXECUTOR_CORES.key, "4") + conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") + conf.set(MEMORY_OFFHEAP_SIZE.key, "3m") conf.set(TASK_GPU_ID.amountConf, "1") conf.set(EXECUTOR_GPU_ID.amountConf, "1") conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, "nameOfScript") val rprof = ResourceProfile.getOrCreateDefaultProfile(conf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val execResources = rprof.executorResources - assert(execResources.size === 5, s"Executor resources should contain cores, pyspark " + - s"memory, memory overhead, memory, and gpu $execResources") + assert(execResources.size === 6, s"Executor resources should contain cores, pyspark " + + s"memory, memory overhead, memory, offHeap memory and gpu $execResources") assert(execResources.contains("gpu"), "Executor resources should have gpu") assert(rprof.executorResources(ResourceProfile.CORES).amount === 4, "Executor resources should have 4 core") @@ -88,6 +92,8 @@ class ResourceProfileSuite extends SparkFunSuite { "pyspark memory empty if not specified") assert(rprof.executorResources(ResourceProfile.OVERHEAD_MEM).amount == 1024, "overhead memory empty if not specified") + assert(rprof.executorResources(ResourceProfile.OFFHEAP_MEM).amount == 3, + "Executor resources should have 3 offHeap memory") assert(rprof.taskResources.size === 2, "Task resources should just contain cpus and gpu") assert(rprof.taskResources.contains("gpu"), "Task resources should have gpu") @@ -172,14 +178,14 @@ class ResourceProfileSuite extends SparkFunSuite { val ereqs = new ExecutorResourceRequests() ereqs.cores(2).memory("4096") - ereqs.memoryOverhead("2048").pysparkMemory("1024") + ereqs.memoryOverhead("2048").pysparkMemory("1024").offHeapMemory("3072") val treqs = new TaskResourceRequests() treqs.cpus(1) rprof.require(treqs) rprof.require(ereqs) - assert(rprof.executorResources.size === 5) + assert(rprof.executorResources.size === 6) assert(rprof.executorResources(ResourceProfile.CORES).amount === 2, "Executor resources should have 2 cores") assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, @@ -188,6 +194,8 @@ class ResourceProfileSuite extends SparkFunSuite { "Executor resources should have 2048 overhead memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 1024, "Executor resources should have 1024 pyspark memory") + assert(rprof.executorResources(ResourceProfile.OFFHEAP_MEM).amount === 3072, + "Executor resources should have 3072 offHeap memory") assert(rprof.taskResources.size === 2) assert(rprof.taskResources("cpus").amount === 1, "Task resources should have cpu") @@ -217,7 +225,7 @@ class ResourceProfileSuite extends SparkFunSuite { val rprof = new ResourceProfileBuilder() val ereqs = new ExecutorResourceRequests() ereqs.memory("4g") - ereqs.memoryOverhead("2000m").pysparkMemory("512000k") + ereqs.memoryOverhead("2000m").pysparkMemory("512000k").offHeapMemory("1g") rprof.require(ereqs) assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, @@ -226,6 +234,8 @@ class ResourceProfileSuite extends SparkFunSuite { "Executor resources should have 2000 overhead memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 500, "Executor resources should have 512 pyspark memory") + assert(rprof.executorResources(ResourceProfile.OFFHEAP_MEM).amount === 1024, + "Executor resources should have 1024 offHeap memory") } test("Test TaskResourceRequest fractional") { @@ -256,4 +266,32 @@ class ResourceProfileSuite extends SparkFunSuite { }.getMessage() assert(taskError.contains("The resource amount 0.7 must be either <= 0.5, or a whole number.")) } + + test("ResourceProfile has correct custom executor resources") { + val rprof = new ResourceProfileBuilder() + val eReq = new ExecutorResourceRequests() + .cores(2).memory("4096") + .memoryOverhead("2048").pysparkMemory("1024").offHeapMemory("3072") + .resource("gpu", 2) + rprof.require(eReq) + + // Update this if new resource type added + assert(ResourceProfile.allSupportedExecutorResources.size === 5, + "Executor resources should have 5 supported resources") + assert(ResourceProfile.getCustomExecutorResources(rprof.build).size === 1, + "Executor resources should have 1 custom resource") + } + + test("ResourceProfile has correct custom task resources") { + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests() + .resource("gpu", 1) + val eReq = new ExecutorResourceRequests() + .cores(2).memory("4096") + .memoryOverhead("2048").pysparkMemory("1024").offHeapMemory("3072") + rprof.require(taskReq).require(eReq) + + assert(ResourceProfile.getCustomTaskResources(rprof.build).size === 1, + "Task resources should have 1 custom resource") + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 7013832757e38..c829006923c4f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -25,6 +25,9 @@ import scala.annotation.meta.param import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.util.control.NonFatal +import org.mockito.Mockito.spy +import org.mockito.Mockito.times +import org.mockito.Mockito.verify import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.exceptions.TestFailedException import org.scalatest.time.SpanSugar._ @@ -169,10 +172,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 - override def executorDecommission(executorId: String) = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None + override def executorDecommission( + executorId: String, + decommissionInfo: ExecutorDecommissionInfo): Unit = {} + override def getExecutorDecommissionInfo( + executorId: String): Option[ExecutorDecommissionInfo] = None } /** @@ -235,6 +242,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi var sparkListener: EventInfoRecordingListener = null + var blockManagerMaster: BlockManagerMaster = null var mapOutputTracker: MapOutputTrackerMaster = null var broadcastManager: BroadcastManager = null var securityMgr: SecurityManager = null @@ -248,17 +256,18 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null, null, conf, true) { - override def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { - blockIds.map { - _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). - getOrElse(Seq()) - }.toIndexedSeq - } - override def removeExecutor(execId: String): Unit = { - // don't need to propagate to the driver, which we don't have - } + class MyBlockManagerMaster(conf: SparkConf) extends BlockManagerMaster(null, null, conf, true) { + override def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { + blockIds.map { + _.asRDDId.map { id => (id.rddId -> id.splitIndex) + }.flatMap { key => cacheLocations.get(key) + }.getOrElse(Seq()) + }.toIndexedSeq } + override def removeExecutor(execId: String): Unit = { + // don't need to propagate to the driver, which we don't have + } + } /** The list of results that DAGScheduler has collected. */ val results = new HashMap[Int, Any]() @@ -276,6 +285,16 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def jobFailed(exception: Exception): Unit = { failure = exception } } + class MyMapOutputTrackerMaster( + conf: SparkConf, + broadcastManager: BroadcastManager) + extends MapOutputTrackerMaster(conf, broadcastManager, true) { + + override def sendTracker(message: Any): Unit = { + // no-op, just so we can stop this to avoid leaking threads + } + } + override def beforeEach(): Unit = { super.beforeEach() init(new SparkConf()) @@ -293,11 +312,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi results.clear() securityMgr = new SecurityManager(conf) broadcastManager = new BroadcastManager(true, conf, securityMgr) - mapOutputTracker = new MapOutputTrackerMaster(conf, broadcastManager, true) { - override def sendTracker(message: Any): Unit = { - // no-op, just so we can stop this to avoid leaking threads - } - } + mapOutputTracker = spy(new MyMapOutputTrackerMaster(conf, broadcastManager)) + blockManagerMaster = spy(new MyBlockManagerMaster(conf)) scheduler = new DAGScheduler( sc, taskScheduler, @@ -548,6 +564,56 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mapStatus2(2).location.host === "hostB") } + test("SPARK-32003: All shuffle files for executor should be cleaned up on fetch failure") { + // reset the test context with the right shuffle service config + afterEach() + val conf = new SparkConf() + conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") + init(conf) + + val shuffleMapRdd = new MyRDD(sc, 3, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(3)) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 3, List(shuffleDep), tracker = mapOutputTracker) + + submit(reduceRdd, Array(0, 1, 2)) + // Map stage completes successfully, + // two tasks are run on an executor on hostA and one on an executor on hostB + completeShuffleMapStageSuccessfully(0, 0, 3, Seq("hostA", "hostA", "hostB")) + // Now the executor on hostA is lost + runEvent(ExecutorLost("hostA-exec", ExecutorExited(-100, false, "Container marked as failed"))) + // Executor is removed but shuffle files are not unregistered + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") + verify(mapOutputTracker, times(0)).removeOutputsOnExecutor("hostA-exec") + + // The MapOutputTracker has all the shuffle files + val mapStatuses = mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses + assert(mapStatuses.count(_ != null) === 3) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostA-exec") === 2) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1) + + // Now a fetch failure from the lost executor occurs + complete(taskSets(1), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null) + )) + // blockManagerMaster.removeExecutor is not called again + // but shuffle files are unregistered + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") + verify(mapOutputTracker, times(1)).removeOutputsOnExecutor("hostA-exec") + + // Shuffle files for hostA-exec should be lost + assert(mapStatuses.count(_ != null) === 1) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostA-exec") === 0) + assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1) + + // Additional fetch failure from the executor does not result in further call to + // mapOutputTracker.removeOutputsOnExecutor + complete(taskSets(1), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 1, 0, "ignored"), null) + )) + verify(mapOutputTracker, times(1)).removeOutputsOnExecutor("hostA-exec") + } + test("zero split job") { var numResults = 0 var failureReason: Option[Exception] = None @@ -715,10 +781,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, executorUpdates: Map[(Int, Int), ExecutorMetrics]): Boolean = true - override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None + override def executorDecommission( + executorId: String, + decommissionInfo: ExecutorDecommissionInfo): Unit = {} + override def getExecutorDecommissionInfo( + executorId: String): Option[ExecutorDecommissionInfo] = None } val noKillScheduler = new DAGScheduler( sc, @@ -765,8 +835,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) - // this will get called - // blockManagerMaster.removeExecutor("hostA-exec") + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") // ask the scheduler to try it again scheduler.resubmitFailedStages() // have the 2nd attempt pass @@ -806,11 +875,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) completeShuffleMapStageSuccessfully(0, 0, 1) runEvent(ExecutorLost("hostA-exec", event)) + verify(blockManagerMaster, times(1)).removeExecutor("hostA-exec") if (expectFileLoss) { + verify(mapOutputTracker, times(1)).removeOutputsOnExecutor("hostA-exec") intercept[MetadataFetchFailedException] { mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) } } else { + verify(mapOutputTracker, times(0)).removeOutputsOnExecutor("hostA-exec") assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) } @@ -2213,7 +2285,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(stackTraceString.contains("org.apache.spark.rdd.RDD.count")) // should include the FunSuite setup: - assert(stackTraceString.contains("org.scalatest.FunSuite")) + assert(stackTraceString.contains("org.scalatest.funsuite.AnyFunSuite")) } test("catch errors in event loop") { @@ -3218,7 +3290,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mergedRp.taskResources.get(GPU).get.amount == 1) val ereqs5 = new ExecutorResourceRequests().cores(1).memory("3g") - .memoryOverhead("1g").pysparkMemory("2g").resource(GPU, 1, "disc") + .memoryOverhead("1g").pysparkMemory("2g").offHeapMemory("4g").resource(GPU, 1, "disc") val treqs5 = new TaskResourceRequests().cpus(1).resource(GPU, 1) val rp5 = new ResourceProfile(ereqs5.requests, treqs5.requests) val ereqs6 = new ExecutorResourceRequests().cores(8).resource(FPGA, 2, "fdisc") @@ -3228,7 +3300,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mergedRp.getTaskCpus.get == 2) assert(mergedRp.getExecutorCores.get == 8) - assert(mergedRp.executorResources.size == 6) + assert(mergedRp.executorResources.size == 7) assert(mergedRp.taskResources.size == 3) assert(mergedRp.executorResources.get(GPU).get.amount == 1) assert(mergedRp.executorResources.get(GPU).get.discoveryScript == "disc") @@ -3239,6 +3311,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mergedRp.executorResources.get(ResourceProfile.MEMORY).get.amount == 3072) assert(mergedRp.executorResources.get(ResourceProfile.PYSPARK_MEM).get.amount == 2048) assert(mergedRp.executorResources.get(ResourceProfile.OVERHEAD_MEM).get.amount == 1024) + assert(mergedRp.executorResources.get(ResourceProfile.OFFHEAP_MEM).get.amount == 4096) val ereqs7 = new ExecutorResourceRequests().cores(1).memory("3g") .resource(GPU, 4, "disc") diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 7ead51bc691fb..07d88672290fc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -90,7 +90,6 @@ private class DummyTaskScheduler extends TaskScheduler { override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = {} override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 - override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None @@ -99,4 +98,9 @@ private class DummyTaskScheduler extends TaskScheduler { accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, executorMetrics: Map[(Int, Int), ExecutorMetrics]): Boolean = true + override def executorDecommission( + executorId: String, + decommissionInfo: ExecutorDecommissionInfo): Unit = {} + override def getExecutorDecommissionInfo( + executorId: String): Option[ExecutorDecommissionInfo] = None } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 270b2c606ad0c..a4a84b0e89809 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -24,7 +24,8 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.mockito.Mockito -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index e43be60e956be..e5836458e7f91 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -1000,6 +1000,42 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!tsm.isZombie) } + test("SPARK-31418 abort timer should kick in when task is completely blacklisted &" + + "allocation manager could not acquire a new executor before the timeout") { + // set the abort timer to fail immediately + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0", + config.DYN_ALLOCATION_ENABLED.key -> "true") + + // We have 2 tasks remaining with 1 executor + val taskSet = FakeTask.createTaskSet(numTasks = 2) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("executor0", "host0", 2))).flatten + + // Fail the running task + failTask(0, TaskState.FAILED, UnknownReason, tsm) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", 0)).thenReturn(true) + + // If the executor is busy, then dynamic allocation should kick in and try + // to acquire additional executors to schedule the blacklisted task + assert(taskScheduler.isExecutorBusy("executor0")) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer to kick in immediately + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + // Wait for the abort timer to kick in. Even though we configure the timeout to be 0, there is a + // slight delay as the abort timer is launched in a separate thread. + eventually(timeout(500.milliseconds)) { + assert(tsm.isZombie) + } + } + /** * Helper for performance tests. Takes the explicitly blacklisted nodes and executors; verifies * that the blacklists are used efficiently to ensure scheduling is not O(numPendingTasks). @@ -1766,6 +1802,53 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(2 == taskDescriptions.head.resources(GPU).addresses.size) } + private def setupSchedulerForDecommissionTests(): TaskSchedulerImpl = { + val taskScheduler = setupSchedulerWithMaster( + s"local[2]", + config.CPUS_PER_TASK.key -> 1.toString) + taskScheduler.submitTasks(FakeTask.createTaskSet(2)) + val multiCoreWorkerOffers = IndexedSeq(WorkerOffer("executor0", "host0", 1), + WorkerOffer("executor1", "host1", 1)) + val taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten + assert(taskDescriptions.map(_.executorId).sorted === Seq("executor0", "executor1")) + taskScheduler + } + + test("scheduler should keep the decommission info where host was decommissioned") { + val scheduler = setupSchedulerForDecommissionTests() + + scheduler.executorDecommission("executor0", ExecutorDecommissionInfo("0", false)) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("1", true)) + scheduler.executorDecommission("executor0", ExecutorDecommissionInfo("0 new", false)) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("1 new", false)) + + assert(scheduler.getExecutorDecommissionInfo("executor0") + === Some(ExecutorDecommissionInfo("0 new", false))) + assert(scheduler.getExecutorDecommissionInfo("executor1") + === Some(ExecutorDecommissionInfo("1", true))) + assert(scheduler.getExecutorDecommissionInfo("executor2").isEmpty) + } + + test("scheduler should ignore decommissioning of removed executors") { + val scheduler = setupSchedulerForDecommissionTests() + + // executor 0 is decommissioned after loosing + assert(scheduler.getExecutorDecommissionInfo("executor0").isEmpty) + scheduler.executorLost("executor0", ExecutorExited(0, false, "normal")) + assert(scheduler.getExecutorDecommissionInfo("executor0").isEmpty) + scheduler.executorDecommission("executor0", ExecutorDecommissionInfo("", false)) + assert(scheduler.getExecutorDecommissionInfo("executor0").isEmpty) + + // executor 1 is decommissioned before loosing + assert(scheduler.getExecutorDecommissionInfo("executor1").isEmpty) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("", false)) + assert(scheduler.getExecutorDecommissionInfo("executor1").isDefined) + scheduler.executorLost("executor1", ExecutorExited(0, false, "normal")) + assert(scheduler.getExecutorDecommissionInfo("executor1").isEmpty) + scheduler.executorDecommission("executor1", ExecutorDecommissionInfo("", false)) + assert(scheduler.getExecutorDecommissionInfo("executor1").isEmpty) + } + /** * Used by tests to simulate a task failure. This calls the failure handler explicitly, to ensure * that all the state is updated when this method returns. Otherwise, there's no way to know when diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 95c8197abbf0b..0a98030a56edc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.Assertions._ import org.scalatest.PrivateMethodTester import org.scalatest.concurrent.Eventually -import org.apache.spark._ +import org.apache.spark.{FakeSchedulerBackend => _, _} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.SKIP_VALIDATE_CORES_TESTING @@ -1911,6 +1911,112 @@ class TaskSetManagerSuite testSpeculationDurationThreshold(true, 2, 1) } + test("SPARK-21040: Check speculative tasks are launched when an executor is decommissioned" + + " and the tasks running on it cannot finish within EXECUTOR_DECOMMISSION_KILL_INTERVAL") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) + val taskSet = FakeTask.createTaskSet(4) + sc.conf.set(config.SPECULATION_ENABLED, true) + sc.conf.set(config.SPECULATION_MULTIPLIER, 1.5) + sc.conf.set(config.SPECULATION_QUANTILE, 0.5) + sc.conf.set(config.EXECUTOR_DECOMMISSION_KILL_INTERVAL.key, "5s") + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => + task.metrics.internalAccums + } + + // Start TASK 0,1 on exec1, TASK 2 on exec2 + (0 until 2).foreach { _ => + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF)._1 + assert(taskOption.isDefined) + assert(taskOption.get.executorId === "exec1") + } + val taskOption2 = manager.resourceOffer("exec2", "host2", NO_PREF)._1 + assert(taskOption2.isDefined) + assert(taskOption2.get.executorId === "exec2") + + clock.advance(6*1000) // time = 6s + // Start TASK 3 on exec2 after some delay + val taskOption3 = manager.resourceOffer("exec2", "host2", NO_PREF)._1 + assert(taskOption3.isDefined) + assert(taskOption3.get.executorId === "exec2") + + assert(sched.startedTasks.toSet === Set(0, 1, 2, 3)) + + clock.advance(4*1000) // time = 10s + // Complete the first 2 tasks and leave the other 2 tasks in running + for (id <- Set(0, 1)) { + manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) + assert(sched.endedTasks(id) === Success) + } + + // checkSpeculatableTasks checks that the task runtime is greater than the threshold for + // speculating. Since we use a SPECULATION_MULTIPLIER of 1.5, So tasks need to be running for + // > 15s for speculation + assert(!manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.toSet === Set()) + + // decommission exec-2. All tasks running on exec-2 (i.e. TASK 2,3) will be added to + // executorDecommissionSpeculationTriggerTimeoutOpt + // (TASK 2 -> 15, TASK 3 -> 15) + manager.executorDecommission("exec2") + assert(manager.tidToExecutorKillTimeMapping.keySet === Set(2, 3)) + assert(manager.tidToExecutorKillTimeMapping(2) === 15*1000) + assert(manager.tidToExecutorKillTimeMapping(3) === 15*1000) + + assert(manager.checkSpeculatableTasks(0)) + // TASK 2 started at t=0s, so it can still finish before t=15s (Median task runtime = 10s) + // TASK 3 started at t=6s, so it might not finish before t=15s. So TASK 3 should be part + // of speculativeTasks + assert(sched.speculativeTasks.toSet === Set(3)) + assert(manager.copiesRunning(3) === 1) + + // Offer resource to start the speculative attempt for the running task + val taskOption3New = manager.resourceOffer("exec3", "host3", NO_PREF)._1 + // Offer more resources. Nothing should get scheduled now. + assert(manager.resourceOffer("exec3", "host3", NO_PREF)._1.isEmpty) + assert(taskOption3New.isDefined) + + // Assert info about the newly launched speculative task + val speculativeTask3 = taskOption3New.get + assert(speculativeTask3.index === 3) + assert(speculativeTask3.taskId === 4) + assert(speculativeTask3.executorId === "exec3") + assert(speculativeTask3.attemptNumber === 1) + + clock.advance(1*1000) // time = 11s + // Running checkSpeculatableTasks again should return false + assert(!manager.checkSpeculatableTasks(0)) + assert(manager.copiesRunning(2) === 1) + assert(manager.copiesRunning(3) === 2) + + clock.advance(5*1000) // time = 16s + // At t=16s, TASK 2 has been running for 16s. It is more than the + // SPECULATION_MULTIPLIER * medianRuntime = 1.5 * 10 = 15s. So now TASK 2 will + // be selected for speculation. Here we are verifying that regular speculation configs + // should still take effect even when a EXECUTOR_DECOMMISSION_KILL_INTERVAL is provided and + // corresponding executor is decommissioned + assert(manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.toSet === Set(2, 3)) + assert(manager.copiesRunning(2) === 1) + assert(manager.copiesRunning(3) === 2) + val taskOption2New = manager.resourceOffer("exec3", "host3", NO_PREF)._1 + assert(taskOption2New.isDefined) + val speculativeTask2 = taskOption2New.get + // Ensure that TASK 2 is re-launched on exec3, host3 + assert(speculativeTask2.index === 2) + assert(speculativeTask2.taskId === 5) + assert(speculativeTask2.executorId === "exec3") + assert(speculativeTask2.attemptNumber === 1) + + assert(manager.copiesRunning(2) === 2) + assert(manager.copiesRunning(3) === 2) + + // Offering additional resources should not lead to any speculative tasks being respawned + assert(manager.resourceOffer("exec1", "host1", ANY)._1.isEmpty) + } + test("SPARK-29976 Regular speculation configs should still take effect even when a " + "threshold is provided") { val (manager, clock) = testSpeculationDurationSetup( diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala index 4de5aaeab5c51..d95deb1f5f327 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionExtendedSuite.scala @@ -65,7 +65,7 @@ class WorkerDecommissionExtendedSuite extends SparkFunSuite with LocalSparkConte val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] sc.getExecutorIds().tail.foreach { id => - sched.decommissionExecutor(id) + sched.decommissionExecutor(id, ExecutorDecommissionInfo("", false)) assert(rdd3.sortByKey().collect().length === 100) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala index 148d20ee659a2..bb0c33acc0af5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -47,7 +47,12 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { assert(sleepyRdd.count() === 10) } - test("verify a task with all workers decommissioned succeeds") { + test("verify a running task with all workers decommissioned succeeds") { + // Wait for the executors to come up + TestUtils.waitUntilExecutorsUp(sc = sc, + numExecutors = 2, + timeout = 30000) // 30s + val input = sc.parallelize(1 to 10) // Listen for the job val sem = new Semaphore(0) @@ -56,9 +61,7 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { sem.release() } }) - TestUtils.waitUntilExecutorsUp(sc = sc, - numExecutors = 2, - timeout = 10000) // 10s + val sleepyRdd = input.mapPartitions{ x => Thread.sleep(5000) // 5s x @@ -73,16 +76,8 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { // decom.sh message passing is tested manually. val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] val execs = sched.getExecutorIds() - execs.foreach(execId => sched.decommissionExecutor(execId)) + execs.foreach(execId => sched.decommissionExecutor(execId, ExecutorDecommissionInfo("", false))) val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 20.seconds) assert(asyncCountResult === 10) - // Try and launch task after decommissioning, this should fail - val postDecommissioned = input.map(x => x) - val postDecomAsyncCount = postDecommissioned.countAsync() - val thrown = intercept[java.util.concurrent.TimeoutException]{ - val result = ThreadUtils.awaitResult(postDecomAsyncCount, 20.seconds) - } - assert(postDecomAsyncCount.isCompleted === false, - "After exec decommission new task could not launch") } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala index fd228cded783a..525e682dd5d42 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -125,7 +125,7 @@ object KryoBenchmark extends BenchmarkBase { def createSerializer(useUnsafe: Boolean): SerializerInstance = { val conf = new SparkConf() conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[MyRegistrator].getName)) conf.set(KRYO_USE_UNSAFE, useUnsafe) new KryoSerializer(conf).newInstance() diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala index 953b651c72a83..dde0c98704d00 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala @@ -76,7 +76,7 @@ object KryoSerializerBenchmark extends BenchmarkBase { conf.set(EXECUTOR_EXTRA_JAVA_OPTIONS, "-XX:+UseParallelGC -XX:-UseDynamicNumberOfGCThreads") conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[MyRegistrator].getName)) conf.set(KRYO_USE_POOL, usePool) if (sc != null) { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index d4fafab4a5d64..397fdce8ae6e3 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -29,7 +29,7 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) .set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - .set(config.Kryo.KRYO_USER_REGISTRATORS, classOf[AppJarRegistrator].getName) + .set(config.Kryo.KRYO_USER_REGISTRATORS, Seq(classOf[AppJarRegistrator].getName)) .set(config.TASK_MAX_FAILURES, 1) .set(config.BLACKLIST_ENABLED, false) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 4c47a67ee9ffc..229ef69973775 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.ThreadUtils class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") - conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[MyRegistrator].getName)) conf.set(KRYO_USE_UNSAFE, false) test("SPARK-7392 configuration limits") { @@ -313,7 +313,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { import org.apache.spark.SparkException val conf = new SparkConf(false) - conf.set(KRYO_USER_REGISTRATORS, "this.class.does.not.exist") + conf.set(KRYO_USER_REGISTRATORS, Seq("this.class.does.not.exist")) val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance().serialize(1)) assert(thrown.getMessage.contains("Failed to register classes with Kryo")) @@ -412,7 +412,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] assert(ser.getAutoReset) val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, - classOf[RegistratorWithoutAutoReset].getName) + Seq(classOf[RegistratorWithoutAutoReset].getName)) val ser2 = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] assert(!ser2.getAutoReset) } @@ -443,7 +443,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { .set(KRYO_REFERENCE_TRACKING, referenceTracking) .set(KRYO_USE_POOL, usePool) if (!autoReset) { - conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[RegistratorWithoutAutoReset].getName)) } val ser = new KryoSerializer(conf) val serInstance = ser.newInstance().asInstanceOf[KryoSerializerInstance] @@ -530,7 +530,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSparkContext { conf.set(SERIALIZER, classOf[KryoSerializer].getName) - conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[RegistratorWithoutAutoReset].getName)) conf.set(KRYO_REFERENCE_TRACKING, true) conf.set(SHUFFLE_MANAGER, "sort") conf.set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 200) diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala index dad080c5fc161..9747f5780dd1e 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala @@ -52,7 +52,7 @@ class SerializerPropertiesSuite extends SparkFunSuite { test("KryoSerializer does not support relocation when auto-reset is disabled") { val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, - classOf[RegistratorWithoutAutoReset].getName) + Seq(classOf[RegistratorWithoutAutoReset].getName)) val ser = new KryoSerializer(conf) assert(!ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) testSupportsRelocationOfSerializedObjects(ser, generateRandomItem) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 27bb06b4e0636..725a1d90557a2 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -27,7 +27,7 @@ import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleBlockInfo} import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -48,6 +48,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(diskBlockManager.getFile(any[BlockId])).thenAnswer( (invocation: InvocationOnMock) => new File(tempDir, invocation.getArguments.head.toString)) + when(diskBlockManager.localDirs).thenReturn(Array(tempDir)) } override def afterEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala index e5f3aab6a6a1a..f75db20808bee 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.shuffle.sort import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index 4c5694fcf0305..c0f5db1892e7e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.shuffle.sort import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} import org.apache.spark.memory.MemoryTestingUtils diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 24eb1685f577a..d5829c352be9b 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1523,14 +1523,32 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.metricToOffset.foreach { metric => - assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) - } + checkExecutorMetrics(metrics, actual) case _ => assert(false) } } } + + // check stage level executor metrics + val expectedStageValues = Map( + 0 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 50L, 10L, 100L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)), + Map( + "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + 1 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 70L, 25L, 40L, 60L, 30L, 70L, 60L, + 40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)), + Map( + "1" -> new ExecutorMetrics(Array(7000L, 70L, 20L, 30L, 60L, 30L, 70L, 55L, + 30L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)), + "2" -> new ExecutorMetrics(Array(5500L, 40L, 25L, 40L, 10L, 30L, 35L, 60L, + 40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) + checkStageExecutorMetrics(expectedStageValues) } test("stage executor metrics") { @@ -1573,14 +1591,74 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.metricToOffset.foreach { metric => - assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) - } + checkExecutorMetrics(metrics, actual) case _ => assert(false) } } } + + // check stage level executor metrics + val expectedStageValues = Map( + 0 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 50L, 10L, 100L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)), + Map( + "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + 1 -> StageExecutorMetrics( + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 60L, 30L, 80L, 60L, + 50L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)), + Map( + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) + checkStageExecutorMetrics(expectedStageValues) + } + + /** expected stage executor metrics */ + private case class StageExecutorMetrics( + peakExecutorMetrics: ExecutorMetrics, + executorMetrics: Map[String, ExecutorMetrics]) + + private def checkExecutorMetrics(expected: ExecutorMetrics, actual: ExecutorMetrics): Unit = { + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === expected.getMetricValue(metric._1)) + } + } + + /** check stage level peak executor metric values, and executor peak values for each stage */ + private def checkStageExecutorMetrics(expectedStageValues: Map[Int, StageExecutorMetrics]) = { + // check stage level peak executor metric values for each stage + for ((stageId, expectedMetrics) <- expectedStageValues) { + check[StageDataWrapper](Array(stageId, 0)) { stage => + stage.info.peakExecutorMetrics match { + case Some(actual) => + checkExecutorMetrics(expectedMetrics.peakExecutorMetrics, actual) + case None => + assert(false) + } + } + } + + // check peak executor metric values for each stage and executor + val stageExecSummaries = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.toSeq + stageExecSummaries.foreach { exec => + expectedStageValues.get(exec.stageId) match { + case Some(stageValue) => + (stageValue.executorMetrics.get(exec.executorId), exec.info.peakMemoryMetrics) match { + case (Some(expected), Some(actual)) => + checkExecutorMetrics(expected, actual) + case _ => + assert(false) + } + case None => + assert(false) + } + } } test("storage information on executor lost/down") { diff --git a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala index 38e88e6a012c4..fec7007279894 100644 --- a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.status import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import org.mockito.Mockito._ -import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config.Status._ diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala index 18baeb1cb9c71..ff2060e278465 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.status.api.v1 import javax.ws.rs.WebApplicationException -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index ef7b13875540f..d7009e6ed5799 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -87,7 +87,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 4) assert(id.mapId === 5) assert(id.reduceId === 6) - assert(!id.isShuffle) + assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } @@ -100,7 +100,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 7) assert(id.mapId === 8) assert(id.reduceId === 9) - assert(!id.isShuffle) + assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala new file mode 100644 index 0000000000000..25145dac52681 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -0,0 +1,287 @@ +/* + * 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.storage + +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, Semaphore, TimeUnit} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ + +import org.scalatest.concurrent.Eventually + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.util.{ResetSystemProperties, SystemClock, ThreadUtils} + +class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalSparkContext + with ResetSystemProperties with Eventually { + + val numExecs = 3 + val numParts = 3 + val TaskStarted = "TASK_STARTED" + val TaskEnded = "TASK_ENDED" + val JobEnded = "JOB_ENDED" + + test(s"verify that an already running task which is going to cache data succeeds " + + s"on a decommissioned executor after task start") { + runDecomTest(true, false, TaskStarted) + } + + test(s"verify that an already running task which is going to cache data succeeds " + + s"on a decommissioned executor after one task ends but before job ends") { + runDecomTest(true, false, TaskEnded) + } + + test(s"verify that shuffle blocks are migrated") { + runDecomTest(false, true, JobEnded) + } + + test(s"verify that both migrations can work at the same time") { + runDecomTest(true, true, JobEnded) + } + + private def runDecomTest( + persist: Boolean, + shuffle: Boolean, + whenToDecom: String): Unit = { + val migrateDuring = whenToDecom != JobEnded + val master = s"local-cluster[${numExecs}, 1, 1024]" + val conf = new SparkConf().setAppName("test").setMaster(master) + .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) + .set(config.STORAGE_DECOMMISSION_ENABLED, true) + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, persist) + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, shuffle) + // Just replicate blocks quickly during testing, there isn't another + // workload we need to worry about. + .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) + + if (whenToDecom == TaskStarted) { + // We are using accumulators below, make sure those are reported frequently. + conf.set(config.EXECUTOR_HEARTBEAT_INTERVAL.key, "10ms") + } + sc = new SparkContext(master, "test", conf) + + // Wait for the executors to start + TestUtils.waitUntilExecutorsUp(sc = sc, + numExecutors = numExecs, + timeout = 60000) // 60s + + val input = sc.parallelize(1 to numParts, numParts) + val accum = sc.longAccumulator("mapperRunAccumulator") + + val sleepIntervalMs = whenToDecom match { + // Increase the window of time b/w task started and ended so that we can decom within that. + case TaskStarted => 2000 + // Make one task take a really short time so that we can decommission right after it is + // done but before its peers are done. + case TaskEnded => + if (TaskContext.getPartitionId() == 0) { + 100 + } else { + 1000 + } + // No sleep otherwise + case _ => 0 + } + + // Create a new RDD where we have sleep in each partition, we are also increasing + // the value of accumulator in each partition + val baseRdd = input.mapPartitions { x => + accum.add(1) + if (sleepIntervalMs > 0) { + Thread.sleep(sleepIntervalMs) + } + x.map(y => (y, y)) + } + val testRdd = shuffle match { + case true => baseRdd.reduceByKey(_ + _) + case false => baseRdd + } + + // Listen for the job & block updates + val executorRemovedSem = new Semaphore(0) + val taskEndEvents = new ConcurrentLinkedQueue[SparkListenerTaskEnd]() + val executorsActuallyStarted = new ConcurrentHashMap[String, Boolean]() + val blocksUpdated = ArrayBuffer.empty[SparkListenerBlockUpdated] + + def getCandidateExecutorToDecom: Option[String] = if (whenToDecom == TaskStarted) { + executorsActuallyStarted.keySet().asScala.headOption + } else { + taskEndEvents.asScala.filter(_.taskInfo.successful).map(_.taskInfo.executorId).headOption + } + + sc.addSparkListener(new SparkListener { + override def onExecutorRemoved(execRemoved: SparkListenerExecutorRemoved): Unit = { + executorRemovedSem.release() + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + taskEndEvents.add(taskEnd) + } + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + blocksUpdated.append(blockUpdated) + } + + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { + val executorId = executorMetricsUpdate.execId + if (executorId != SparkContext.DRIVER_IDENTIFIER) { + val validUpdate = executorMetricsUpdate + .accumUpdates + .flatMap(_._4) + .exists { accumInfo => + accumInfo.name == accum.name && accumInfo.update.exists(_.asInstanceOf[Long] >= 1) + } + if (validUpdate) { + executorsActuallyStarted.put(executorId, java.lang.Boolean.TRUE) + } + } + } + }) + + // Cache the RDD lazily + if (persist) { + testRdd.persist() + } + + // Start the computation of RDD - this step will also cache the RDD + val asyncCount = testRdd.countAsync() + + // Make sure the job is either mid run or otherwise has data to migrate. + if (migrateDuring) { + // Wait for one of the tasks to succeed and finish writing its blocks. + // This way we know that this executor had real data to migrate when it is subsequently + // decommissioned below. + val intervalMs = if (whenToDecom == TaskStarted) { + 3.milliseconds + } else { + 10.milliseconds + } + eventually(timeout(6.seconds), interval(intervalMs)) { + assert(getCandidateExecutorToDecom.isDefined) + } + } else { + ThreadUtils.awaitResult(asyncCount, 15.seconds) + } + + // Decommission one of the executors. + val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] + + val execToDecommission = getCandidateExecutorToDecom.get + logInfo(s"Decommissioning executor ${execToDecommission}") + sched.decommissionExecutor( + execToDecommission, + ExecutorDecommissionInfo("", isHostDecommissioned = false)) + val decomTime = new SystemClock().getTimeMillis() + + // Wait for job to finish. + val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 15.seconds) + assert(asyncCountResult === numParts) + // All tasks finished, so accum should have been increased numParts times. + assert(accum.value === numParts) + + sc.listenerBus.waitUntilEmpty() + val taskEndEventsCopy = taskEndEvents.asScala + if (shuffle) { + // mappers & reducers which succeeded + assert(taskEndEventsCopy.count(_.reason == Success) === 2 * numParts, + s"Expected ${2 * numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})") + } else { + // only mappers which executed successfully + assert(taskEndEventsCopy.count(_.reason == Success) === numParts, + s"Expected ${numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})") + } + + val minTaskEndTime = taskEndEventsCopy.map(_.taskInfo.finishTime).min + val maxTaskEndTime = taskEndEventsCopy.map(_.taskInfo.finishTime).max + + // Verify that the decom time matched our expectations + val decomAssertMsg = s"$whenToDecom: decomTime: $decomTime, minTaskEnd: $minTaskEndTime," + + s" maxTaskEnd: $maxTaskEndTime" + assert(minTaskEndTime <= maxTaskEndTime, decomAssertMsg) + whenToDecom match { + case TaskStarted => assert(minTaskEndTime > decomTime, decomAssertMsg) + case TaskEnded => assert(minTaskEndTime <= decomTime && + decomTime < maxTaskEndTime, decomAssertMsg) + case JobEnded => assert(maxTaskEndTime <= decomTime, decomAssertMsg) + } + + // Wait for our respective blocks to have migrated + eventually(timeout(30.seconds), interval(10.milliseconds)) { + if (persist) { + // One of our blocks should have moved. + val rddUpdates = blocksUpdated.filter { update => + val blockId = update.blockUpdatedInfo.blockId + blockId.isRDD} + val blockLocs = rddUpdates.map { update => + (update.blockUpdatedInfo.blockId.name, + update.blockUpdatedInfo.blockManagerId)} + val blocksToManagers = blockLocs.groupBy(_._1).mapValues(_.size) + assert(!blocksToManagers.filter(_._2 > 1).isEmpty, + s"We should have a block that has been on multiple BMs in rdds:\n ${rddUpdates} from:\n" + + s"${blocksUpdated}\n but instead we got:\n ${blocksToManagers}") + } + // If we're migrating shuffles we look for any shuffle block updates + // as there is no block update on the initial shuffle block write. + if (shuffle) { + val numDataLocs = blocksUpdated.filter { update => + val blockId = update.blockUpdatedInfo.blockId + blockId.isInstanceOf[ShuffleDataBlockId] + }.size + val numIndexLocs = blocksUpdated.filter { update => + val blockId = update.blockUpdatedInfo.blockId + blockId.isInstanceOf[ShuffleIndexBlockId] + }.size + assert(numDataLocs === 1, s"Expect shuffle data block updates in ${blocksUpdated}") + assert(numIndexLocs === 1, s"Expect shuffle index block updates in ${blocksUpdated}") + } + } + + // Since the RDD is cached or shuffled so further usage of same RDD should use the + // cached data. Original RDD partitions should not be recomputed i.e. accum + // should have same value like before + assert(testRdd.count() === numParts) + assert(accum.value === numParts) + + val storageStatus = sc.env.blockManager.master.getStorageStatus + val execIdToBlocksMapping = storageStatus.map( + status => (status.blockManagerId.executorId, status.blocks)).toMap + // No cached blocks should be present on executor which was decommissioned + assert( + !execIdToBlocksMapping.contains(execToDecommission) || + execIdToBlocksMapping(execToDecommission).keys.filter(_.isRDD).toSeq === Seq(), + "Cache blocks should be migrated") + if (persist) { + // There should still be all the RDD blocks cached + assert(execIdToBlocksMapping.values.flatMap(_.keys).count(_.isRDD) === numParts) + } + + // Wait for the executor to be removed automatically after migration. + assert(executorRemovedSem.tryAcquire(1, 5L, TimeUnit.MINUTES)) + + // Since the RDD is cached or shuffled so further usage of same RDD should use the + // cached data. Original RDD partitions should not be recomputed i.e. accum + // should have same value like before + assert(testRdd.count() === numParts) + assert(accum.value === numParts) + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala deleted file mode 100644 index 7456ca7f02a2e..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala +++ /dev/null @@ -1,106 +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.storage - -import java.util.concurrent.Semaphore - -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration._ - -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite, Success} -import org.apache.spark.internal.config -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend -import org.apache.spark.util.{ResetSystemProperties, ThreadUtils} - -class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext - with ResetSystemProperties { - - override def beforeEach(): Unit = { - val conf = new SparkConf().setAppName("test") - .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) - .set(config.STORAGE_DECOMMISSION_ENABLED, true) - - sc = new SparkContext("local-cluster[2, 1, 1024]", "test", conf) - } - - test(s"verify that an already running task which is going to cache data succeeds " + - s"on a decommissioned executor") { - // Create input RDD with 10 partitions - val input = sc.parallelize(1 to 10, 10) - val accum = sc.longAccumulator("mapperRunAccumulator") - // Do a count to wait for the executors to be registered. - input.count() - - // Create a new RDD where we have sleep in each partition, we are also increasing - // the value of accumulator in each partition - val sleepyRdd = input.mapPartitions { x => - Thread.sleep(500) - accum.add(1) - x - } - - // Listen for the job - val sem = new Semaphore(0) - val taskEndEvents = ArrayBuffer.empty[SparkListenerTaskEnd] - sc.addSparkListener(new SparkListener { - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - sem.release() - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - taskEndEvents.append(taskEnd) - } - }) - - // Cache the RDD lazily - sleepyRdd.persist() - - // Start the computation of RDD - this step will also cache the RDD - val asyncCount = sleepyRdd.countAsync() - - // Wait for the job to have started - sem.acquire(1) - - // Give Spark a tiny bit to start the tasks after the listener says hello - Thread.sleep(100) - // Decommission one of the executor - val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] - val execs = sched.getExecutorIds() - assert(execs.size == 2, s"Expected 2 executors but found ${execs.size}") - val execToDecommission = execs.head - sched.decommissionExecutor(execToDecommission) - - // Wait for job to finish - val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 6.seconds) - assert(asyncCountResult === 10) - // All 10 tasks finished, so accum should have been increased 10 times - assert(accum.value === 10) - - // All tasks should be successful, nothing should have failed - sc.listenerBus.waitUntilEmpty() - assert(taskEndEvents.size === 10) // 10 mappers - assert(taskEndEvents.map(_.reason).toSet === Set(Success)) - - // Since the RDD is cached, so further usage of same RDD should use the - // cached data. Original RDD partitions should not be recomputed i.e. accum - // should have same value like before - assert(sleepyRdd.count() === 10) - assert(accum.value === 10) - } -} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala new file mode 100644 index 0000000000000..74ad8bd2bcf9d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -0,0 +1,225 @@ +/* + * 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.storage + +import scala.concurrent.duration._ + +import org.mockito.{ArgumentMatchers => mc} +import org.mockito.Mockito.{atLeast => least, mock, times, verify, when} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.shuffle.{MigratableResolver, ShuffleBlockInfo} +import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock + +class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { + + private val bmPort = 12345 + + private val sparkConf = new SparkConf(false) + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, true) + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, true) + // Just replicate blocks quickly during testing, as there isn't another + // workload we need to worry about. + .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) + + private def registerShuffleBlocks( + mockMigratableShuffleResolver: MigratableResolver, + ids: Set[(Int, Long, Int)]): Unit = { + + when(mockMigratableShuffleResolver.getStoredShuffles()) + .thenReturn(ids.map(triple => ShuffleBlockInfo(triple._1, triple._2)).toSeq) + + ids.foreach { case (shuffleId: Int, mapId: Long, reduceId: Int) => + when(mockMigratableShuffleResolver.getMigrationBlocks(mc.any())) + .thenReturn(List( + (ShuffleIndexBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer])), + (ShuffleDataBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer])))) + } + } + + /** + * Validate a given configuration with the mocks. + * The fail variable controls if we expect migration to fail, in which case we expect + * a constant Long.MaxValue timestamp. + */ + private def validateDecommissionTimestamps(conf: SparkConf, bm: BlockManager, + migratableShuffleBlockResolver: MigratableResolver, fail: Boolean = false) = { + // Verify the decommissioning manager timestamps and status + val bmDecomManager = new BlockManagerDecommissioner(conf, bm) + var previousTime: Option[Long] = None + try { + bmDecomManager.start() + eventually(timeout(100.second), interval(10.milliseconds)) { + val (currentTime, done) = bmDecomManager.lastMigrationInfo() + assert(done) + // Make sure the time stamp starts moving forward. + if (!fail) { + previousTime match { + case None => + previousTime = Some(currentTime) + assert(false) + case Some(t) => + assert(t < currentTime) + } + } else { + // If we expect migration to fail we should get the max value quickly. + assert(currentTime === Long.MaxValue) + } + } + if (!fail) { + // Wait 5 seconds and assert times keep moving forward. + Thread.sleep(5000) + val (currentTime, done) = bmDecomManager.lastMigrationInfo() + assert(done && currentTime > previousTime.get) + } + } finally { + bmDecomManager.stop() + } + } + + test("test that with no blocks we finish migration") { + // Set up the mocks so we return empty + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + when(migratableShuffleBlockResolver.getStoredShuffles()) + .thenReturn(Seq()) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(sparkConf, bm, migratableShuffleBlockResolver) + } + + test("block decom manager with no migrations configured") { + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + val badConf = new SparkConf(false) + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, false) + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, false) + .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(badConf, bm, migratableShuffleBlockResolver, + fail = true) + } + + test("block decom manager with no peers") { + // Set up the mocks so we return one shuffle block + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq()) + + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(sparkConf, bm, migratableShuffleBlockResolver, + fail = true) + } + + + test("block decom manager with only shuffle files time moves forward") { + // Set up the mocks so we return one shuffle block + val bm = mock(classOf[BlockManager]) + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq()) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + // Verify the decom manager handles this correctly + validateDecommissionTimestamps(sparkConf, bm, migratableShuffleBlockResolver) + } + + test("test shuffle and cached rdd migration without any error") { + val blockTransferService = mock(classOf[BlockTransferService]) + val bm = mock(classOf[BlockManager]) + + val storedBlockId1 = RDDBlockId(0, 0) + val storedBlock1 = + new ReplicateBlock(storedBlockId1, Seq(BlockManagerId("replicaHolder", "host1", bmPort)), 1) + + val migratableShuffleBlockResolver = mock(classOf[MigratableResolver]) + registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1))) + when(bm.getPeers(mc.any())) + .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345))) + + when(bm.blockTransferService).thenReturn(blockTransferService) + when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver) + when(bm.getMigratableRDDBlocks()) + .thenReturn(Seq(storedBlock1)) + + val bmDecomManager = new BlockManagerDecommissioner(sparkConf, bm) + + try { + bmDecomManager.start() + + var previousRDDTime: Option[Long] = None + var previousShuffleTime: Option[Long] = None + + // We don't check that all blocks are migrated because out mock is always returning an RDD. + eventually(timeout(100.second), interval(10.milliseconds)) { + assert(bmDecomManager.shufflesToMigrate.isEmpty == true) + verify(bm, least(1)).replicateBlock( + mc.eq(storedBlockId1), mc.any(), mc.any(), mc.eq(Some(3))) + verify(blockTransferService, times(2)) + .uploadBlockSync(mc.eq("host2"), mc.eq(bmPort), mc.eq("exec2"), mc.any(), mc.any(), + mc.eq(StorageLevel.DISK_ONLY), mc.isNull()) + // Since we never "finish" the RDD blocks, make sure the time is always moving forward. + assert(bmDecomManager.rddBlocksLeft) + previousRDDTime match { + case None => + previousRDDTime = Some(bmDecomManager.lastRDDMigrationTime) + assert(false) + case Some(t) => + assert(bmDecomManager.lastRDDMigrationTime > t) + } + // Since we do eventually finish the shuffle blocks make sure the shuffle blocks complete + // and that the time keeps moving forward. + assert(!bmDecomManager.shuffleBlocksLeft) + previousShuffleTime match { + case None => + previousShuffleTime = Some(bmDecomManager.lastShuffleMigrationTime) + assert(false) + case Some(t) => + assert(bmDecomManager.lastShuffleMigrationTime > t) + } + } + } finally { + bmDecomManager.stop() + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 660bfcfc48267..0b673c580d71f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -26,8 +26,9 @@ import scala.language.implicitConversions import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{doAnswer, mock, spy, when} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager @@ -103,7 +104,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), + new LiveListenerBus(conf), None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) allStores.clear() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index dc1c7cd52d466..2a3d2d7f86a7e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.File import java.nio.ByteBuffer +import java.nio.file.Files import scala.collection.JavaConverters._ import scala.collection.mutable @@ -31,9 +32,11 @@ import scala.reflect.ClassTag import org.apache.commons.lang3.RandomUtils import org.mockito.{ArgumentCaptor, ArgumentMatchers => mc} import org.mockito.Mockito.{doAnswer, mock, never, spy, times, verify, when} -import org.scalatest._ +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager @@ -50,10 +53,11 @@ import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, Transpo import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExecutorDiskUtils, ExternalBlockStoreClient} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv} -import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerBlockUpdated} +import org.apache.spark.scheduler.{LiveListenerBus, MapStatus, SparkListenerBlockUpdated} import org.apache.spark.scheduler.cluster.{CoarseGrainedClusterMessages, CoarseGrainedSchedulerBackend} import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.{ShuffleBlockResolver, ShuffleManager} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util._ @@ -61,7 +65,7 @@ import org.apache.spark.util.io.ChunkedByteBuffer class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach with PrivateMethodTester with LocalSparkContext with ResetSystemProperties - with EncryptionFunSuite with TimeLimits { + with EncryptionFunSuite with TimeLimits with BeforeAndAfterAll { import BlockManagerSuite._ @@ -70,6 +74,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE var conf: SparkConf = null val allStores = ArrayBuffer[BlockManager]() + val sortShuffleManagers = ArrayBuffer[SortShuffleManager]() var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null var liveListenerBus: LiveListenerBus = null @@ -97,12 +102,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .set(Network.RPC_ASK_TIMEOUT, "5s") } + private def makeSortShuffleManager(): SortShuffleManager = { + val newMgr = new SortShuffleManager(new SparkConf(false)) + sortShuffleManagers += newMgr + newMgr + } + private def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master, transferService: Option[BlockTransferService] = Option.empty, - testConf: Option[SparkConf] = None): BlockManager = { + testConf: Option[SparkConf] = None, + shuffleManager: ShuffleManager = shuffleManager): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) bmConf.set(TEST_MEMORY, maxMem) bmConf.set(MEMORY_OFFHEAP_SIZE, maxMem) @@ -153,7 +165,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE liveListenerBus = spy(new LiveListenerBus(conf)) master = spy(new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - liveListenerBus, None, blockManagerInfo)), + liveListenerBus, None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true)) @@ -166,6 +178,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf = null allStores.foreach(_.stop()) allStores.clear() + sortShuffleManagers.foreach(_.stop()) + sortShuffleManagers.clear() rpcEnv.shutdown() rpcEnv.awaitTermination() rpcEnv = null @@ -176,6 +190,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } + override def afterAll(): Unit = { + try { + // Cleanup the reused items. + Option(bcastManager).foreach(_.stop()) + Option(mapOutputTracker).foreach(_.stop()) + Option(shuffleManager).foreach(_.stop()) + } finally { + super.afterAll() + } + } + private def stopBlockManager(blockManager: BlockManager): Unit = { allStores -= blockManager blockManager.stop() @@ -1815,13 +1840,26 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE verify(liveListenerBus, never()).post(SparkListenerBlockUpdated(BlockUpdatedInfo(updateInfo))) } + test("we reject putting blocks when we have the wrong shuffle resolver") { + val badShuffleManager = mock(classOf[ShuffleManager]) + val badShuffleResolver = mock(classOf[ShuffleBlockResolver]) + when(badShuffleManager.shuffleBlockResolver).thenReturn(badShuffleResolver) + val shuffleBlockId = ShuffleDataBlockId(0, 0, 0) + val bm = makeBlockManager(100, "exec1", shuffleManager = badShuffleManager) + val message = "message" + val exception = intercept[SparkException] { + bm.putBlockDataAsStream(shuffleBlockId, StorageLevel.DISK_ONLY, ClassTag(message.getClass)) + } + assert(exception.getMessage.contains("unsupported shuffle resolver")) + } + test("test decommission block manager should not be part of peers") { val exec1 = "exec1" val exec2 = "exec2" val exec3 = "exec3" - val store1 = makeBlockManager(800, exec1) - val store2 = makeBlockManager(800, exec2) - val store3 = makeBlockManager(800, exec3) + val store1 = makeBlockManager(1000, exec1) + val store2 = makeBlockManager(1000, exec2) + val store3 = makeBlockManager(1000, exec3) assert(master.getPeers(store3.blockManagerId).map(_.executorId).toSet === Set(exec1, exec2)) @@ -1836,9 +1874,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("test decommissionRddCacheBlocks should offload all cached blocks") { - val store1 = makeBlockManager(800, "exec1") - val store2 = makeBlockManager(800, "exec2") - val store3 = makeBlockManager(800, "exec3") + val store1 = makeBlockManager(1000, "exec1") + val store2 = makeBlockManager(1000, "exec2") + val store3 = makeBlockManager(1000, "exec3") val data = new Array[Byte](4) val blockId = rdd(0, 0) @@ -1846,7 +1884,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).contains(store1.blockManagerId)) - store1.decommissionRddCacheBlocks() + val decomManager = new BlockManagerDecommissioner(conf, store1) + decomManager.decommissionRddCacheBlocks() assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).toSet === Set(store2.blockManagerId, store3.blockManagerId)) @@ -1866,13 +1905,57 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) assert(master.getLocations(blockIdSmall) === Seq(store1.blockManagerId)) - store1.decommissionRddCacheBlocks() + val decomManager = new BlockManagerDecommissioner(conf, store1) + decomManager.decommissionRddCacheBlocks() // Smaller block offloaded to store2 assert(master.getLocations(blockIdSmall) === Seq(store2.blockManagerId)) // Larger block still present in store1 as it can't be offloaded assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) } + test("test migration of shuffle blocks during decommissioning") { + val shuffleManager1 = makeSortShuffleManager() + val bm1 = makeBlockManager(3500, "exec1", shuffleManager = shuffleManager1) + shuffleManager1.shuffleBlockResolver._blockManager = bm1 + + val shuffleManager2 = makeSortShuffleManager() + val bm2 = makeBlockManager(3500, "exec2", shuffleManager = shuffleManager2) + shuffleManager2.shuffleBlockResolver._blockManager = bm2 + + val blockSize = 5 + val shuffleDataBlockContent = Array[Byte](0, 1, 2, 3, 4) + val shuffleData = ShuffleDataBlockId(0, 0, 0) + Files.write(bm1.diskBlockManager.getFile(shuffleData).toPath(), shuffleDataBlockContent) + val shuffleIndexBlockContent = Array[Byte](5, 6, 7, 8, 9) + val shuffleIndex = ShuffleIndexBlockId(0, 0, 0) + Files.write(bm1.diskBlockManager.getFile(shuffleIndex).toPath(), shuffleIndexBlockContent) + + mapOutputTracker.registerShuffle(0, 1) + val decomManager = new BlockManagerDecommissioner(conf, bm1) + try { + mapOutputTracker.registerMapOutput(0, 0, MapStatus(bm1.blockManagerId, Array(blockSize), 0)) + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm1.blockManagerId) + + val env = mock(classOf[SparkEnv]) + when(env.conf).thenReturn(conf) + SparkEnv.set(env) + + decomManager.refreshOffloadingShuffleBlocks() + + eventually(timeout(1.second), interval(10.milliseconds)) { + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm2.blockManagerId) + } + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleData).toPath()) + === shuffleDataBlockContent) + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleIndex).toPath()) + === shuffleIndexBlockContent) + } finally { + mapOutputTracker.unregisterShuffle(0) + // Avoid thread leak + decomManager.stopOffloadingShuffleBlocks() + } + } + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { var numCalls = 0 var tempFileManager: DownloadFileManager = null diff --git a/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala index 4000218e71a8b..9ce4acc75ec43 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala @@ -21,7 +21,8 @@ import scala.collection.mutable import scala.language.implicitConversions import scala.util.Random -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers import org.apache.spark.{LocalSparkContext, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala index 0bc26adeeb443..03e27bfaf2d45 100644 --- a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.storage import java.io.{File, FileOutputStream} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.internal.config.STORAGE_REPLICATION_TOPOLOGY_FILE diff --git a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala index 4b018f69b1660..2c135850b5a21 100644 --- a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.ui import org.openqa.selenium.{By, WebDriver} -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 3d52199b01327..48e0d218c0e5c 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -93,7 +93,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { tasks = None, executorSummary = None, killedTasksSummary = Map.empty, - ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, + peakExecutorMetrics = None ) val taskTable = new TaskPagedTable( stageData, diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 4f808f03e5dab..6817fccd10305 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -30,8 +30,10 @@ import org.json4s._ import org.json4s.jackson.JsonMethods import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2ad4a634cd9a7..56026eaa0072b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -262,6 +262,27 @@ class UISuite extends SparkFunSuite { } } + test("SPARK-32467: Avoid encoding URL twice on https redirect") { + val (conf, securityMgr, sslOptions) = sslEnabledConf() + val serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, conf) + try { + val serverAddr = s"http://localhost:${serverInfo.boundPort}" + + val (_, ctx) = newContext("/ctx1") + serverInfo.addHandler(ctx, securityMgr) + + TestUtils.withHttpConnection(new URL(s"$serverAddr/ctx%281%29?a%5B0%5D=b")) { conn => + assert(conn.getResponseCode() === HttpServletResponse.SC_FOUND) + val location = Option(conn.getHeaderFields().get("Location")) + .map(_.get(0)).orNull + val expectedLocation = s"https://localhost:${serverInfo.securePort.get}/ctx(1)?a[0]=b" + assert(location == expectedLocation) + } + } finally { + stopServer(serverInfo) + } + } + test("http -> https redirect applies to all URIs") { val (conf, securityMgr, sslOptions) = sslEnabledConf() val serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, conf) diff --git a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala index d3a95e399c289..0ffa9842e16f5 100644 --- a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.util -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c75e98f39758d..f3ed233da7122 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1111,20 +1111,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1170,20 +1170,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1694,20 +1694,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1757,20 +1757,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1837,20 +1837,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 @@ -1934,20 +1934,20 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Details": "details", | "Accumulables": [ | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { | "ID": 1, | "Name": "Accumulable1", | "Update": "delta1", | "Value": "val1", | "Internal": false, | "Count Failed Values": false + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false | } | ], | "Resource Profile Id" : 0 diff --git a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala index 597e0b9597c38..3063e79704fff 100644 --- a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -21,7 +21,8 @@ import java.net.URLClassLoader import scala.collection.JavaConverters._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkContext, SparkException, SparkFunSuite, TestUtils} diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index 1efd399b5db68..4909d5f8325cc 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -21,7 +21,8 @@ import java.util.NoSuchElementException import scala.collection.mutable.Buffer -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadRunnerSuite.scala b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadRunnerSuite.scala new file mode 100644 index 0000000000000..40312beacdff6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadRunnerSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import org.apache.spark.SparkFunSuite + +class UninterruptibleThreadRunnerSuite extends SparkFunSuite { + private var runner: UninterruptibleThreadRunner = null + + override def beforeEach(): Unit = { + runner = new UninterruptibleThreadRunner("ThreadName") + } + + override def afterEach(): Unit = { + runner.shutdown() + } + + test("runUninterruptibly should switch to UninterruptibleThread") { + assert(!Thread.currentThread().isInstanceOf[UninterruptibleThread]) + var isUninterruptibleThread = false + runner.runUninterruptibly { + isUninterruptibleThread = Thread.currentThread().isInstanceOf[UninterruptibleThread] + } + assert(isUninterruptibleThread, "The runner task must run in UninterruptibleThread") + } + + test("runUninterruptibly should not add new UninterruptibleThread") { + var isInitialUninterruptibleThread = false + var isRunnerUninterruptibleThread = false + val t = new UninterruptibleThread("test") { + override def run(): Unit = { + runUninterruptibly { + val initialThread = Thread.currentThread() + isInitialUninterruptibleThread = initialThread.isInstanceOf[UninterruptibleThread] + runner.runUninterruptibly { + val runnerThread = Thread.currentThread() + isRunnerUninterruptibleThread = runnerThread.isInstanceOf[UninterruptibleThread] + assert(runnerThread.eq(initialThread)) + } + } + } + } + t.start() + t.join() + assert(isInitialUninterruptibleThread, + "The initiator must already run in UninterruptibleThread") + assert(isRunnerUninterruptibleThread, "The runner task must run in UninterruptibleThread") + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 0b4e1494bf300..83595ba22aa57 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -21,8 +21,8 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.ref.WeakReference -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.internal.config._ diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 68bcc5e5a5092..08fed93364060 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.SizeEstimator diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index 44d2118d77945..89a308556d5df 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.util.collection -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.SizeEstimator diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala index f5ee428020fd4..3b42300ccdd6a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.SizeEstimator diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index bb03f0d3cdc20..82b4f2eac6248 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -39,8 +39,8 @@ class SorterSuite extends SparkFunSuite with Logging { new Sorter(new KeyReuseIntArraySortDataFormat) .sort(data2, 0, data2.length, Ordering[IntWrapper]) - assert(data0.view === data1.view) - assert(data0.view === data2.view) + assert(data0 === data1) + assert(data0 === data2) } test("KVArraySorter") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 9ae6a8ef879f3..b3e5e0a73dd9f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -157,7 +157,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { buffer, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val result = collectToArray(buffer, outOffset, N) - assert(ref.view == result.view) + assert(ref === result) } test("sort key prefix " + sortType.name) { @@ -169,7 +169,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) - assert(res1.view == res2.view) + assert(res1 === res2) } fuzzTest(s"fuzz test ${sortType.name} with random bitmasks") { seed => @@ -181,7 +181,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { buffer, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val result = collectToArray(buffer, outOffset, N) - assert(ref.view == result.view) + assert(ref === result) } fuzzTest(s"fuzz test key prefix ${sortType.name} with random bitmasks") { seed => @@ -194,7 +194,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) - assert(res1.view == res2.view) + assert(res1 ===res2) } } } diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index fef514e0c4e37..b3f54ff186194 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -22,7 +22,8 @@ import java.util.Random import scala.collection.mutable.ArrayBuffer import org.apache.commons.math3.distribution.PoissonDistribution -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index df3483830ca9c..267ab1020a185 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.util.random import org.apache.commons.math3.stat.inference.ChiSquareTest -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.util.Utils.times diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 326e561529073..df1dd51a7c519 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -47,9 +47,8 @@ jsonFormatter.min.js .*json .*data .*log -pyspark-coverage-site/ -cloudpickle.py -heapq3.py +pyspark-coverage-site/* +cloudpickle/* join.py SparkExprTyper.scala SparkILoop.scala @@ -124,3 +123,4 @@ SessionHandler.java GangliaReporter.java application_1578436911597_0052 config.properties +app-20200706201101-0003 diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index d9135173419ae..57775dde9dd67 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 241b7ed539ae9..9f1dffbd09ac4 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 44d602415b262..a02a6b7bccf27 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -33,7 +33,10 @@ ENV DEBCONF_NONINTERACTIVE_SEEN true # These arguments are just for reuse and not really meant to be customized. ARG APT_INSTALL="apt-get install --no-install-recommends -y" -ARG PIP_PKGS="sphinx==2.3.1 mkdocs==1.0.4 numpy==1.18.1" +# TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. +# See also https://github.com/sphinx-doc/sphinx/issues/7551. +# We should use the latest Sphinx version once this is fixed. +ARG PIP_PKGS="sphinx==3.0.4 mkdocs==1.0.4 numpy==1.18.1 pydata_sphinx_theme==0.3.1" ARG GEM_PKGS="jekyll:4.0.0 jekyll-redirect-from:0.16.0 rouge:3.15.0" # Install extra needed repos and refresh. diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py index be30e6ad30b24..554acc8b8a379 100755 --- a/dev/create-release/translate-contributors.py +++ b/dev/create-release/translate-contributors.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 344806e447689..4936d808db85a 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -1,7 +1,7 @@ JLargeArrays/1.5//JLargeArrays-1.5.jar JTransforms/3.1//JTransforms-3.1.jar JavaEWAH/0.3.2//JavaEWAH-0.3.2.jar -RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +RoaringBitmap/0.9.0//RoaringBitmap-0.9.0.jar ST4/4.0.4//ST4-4.0.4.jar activation/1.1.1//activation-1.1.1.jar aircompressor/0.10//aircompressor-0.10.jar @@ -129,10 +129,10 @@ jline/2.14.6//jline-2.14.6.jar joda-time/2.10.5//joda-time-2.10.5.jar jodd-core/3.5.2//jodd-core-3.5.2.jar jpam/1.1//jpam-1.1.jar -json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar -json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +json4s-ast_2.12/3.7.0-M5//json4s-ast_2.12-3.7.0-M5.jar +json4s-core_2.12/3.7.0-M5//json4s-core_2.12-3.7.0-M5.jar +json4s-jackson_2.12/3.7.0-M5//json4s-jackson_2.12-3.7.0-M5.jar +json4s-scalap_2.12/3.7.0-M5//json4s-scalap_2.12-3.7.0-M5.jar jsp-api/2.1//jsp-api-2.1.jar jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar @@ -156,10 +156,10 @@ metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar metrics-json/4.1.1//metrics-json-4.1.1.jar metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar -objenesis/2.5.1//objenesis-2.5.1.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar +objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar -okio/1.15.0//okio-1.15.0.jar +okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar orc-core/1.5.10/nohive/orc-core-1.5.10-nohive.jar orc-mapreduce/1.5.10/nohive/orc-mapreduce-1.5.10-nohive.jar @@ -184,7 +184,7 @@ scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar scala-reflect/2.12.10//scala-reflect-2.12.10.jar scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar -shims/0.7.45//shims-0.7.45.jar +shims/0.9.0//shims-0.9.0.jar slf4j-api/1.7.30//slf4j-api-1.7.30.jar slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar snakeyaml/1.24//snakeyaml-1.24.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 969249b963e7b..82cd20bf77191 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -1,7 +1,7 @@ HikariCP/2.5.1//HikariCP-2.5.1.jar JLargeArrays/1.5//JLargeArrays-1.5.jar JTransforms/3.1//JTransforms-3.1.jar -RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +RoaringBitmap/0.9.0//RoaringBitmap-0.9.0.jar ST4/4.0.4//ST4-4.0.4.jar activation/1.1.1//activation-1.1.1.jar aircompressor/0.10//aircompressor-0.10.jar @@ -144,10 +144,10 @@ joda-time/2.10.5//joda-time-2.10.5.jar jodd-core/3.5.2//jodd-core-3.5.2.jar jpam/1.1//jpam-1.1.jar json/1.8//json-1.8.jar -json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar -json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +json4s-ast_2.12/3.7.0-M5//json4s-ast_2.12-3.7.0-M5.jar +json4s-core_2.12/3.7.0-M5//json4s-core_2.12-3.7.0-M5.jar +json4s-jackson_2.12/3.7.0-M5//json4s-jackson_2.12-3.7.0-M5.jar +json4s-scalap_2.12/3.7.0-M5//json4s-scalap_2.12-3.7.0-M5.jar jsp-api/2.1//jsp-api-2.1.jar jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar @@ -171,10 +171,10 @@ metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar metrics-json/4.1.1//metrics-json-4.1.1.jar metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar -objenesis/2.5.1//objenesis-2.5.1.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar +objenesis/2.6//objenesis-2.6.jar okhttp/3.12.6//okhttp-3.12.6.jar -okio/1.15.0//okio-1.15.0.jar +okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar orc-core/1.5.10//orc-core-1.5.10.jar orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar @@ -198,7 +198,7 @@ scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar scala-reflect/2.12.10//scala-reflect-2.12.10.jar scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar -shims/0.7.45//shims-0.7.45.jar +shims/0.9.0//shims-0.9.0.jar slf4j-api/1.7.30//slf4j-api-1.7.30.jar slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar snakeyaml/1.24//snakeyaml-1.24.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index e98e4676107ed..7257fb8722422 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -1,7 +1,7 @@ HikariCP/2.5.1//HikariCP-2.5.1.jar JLargeArrays/1.5//JLargeArrays-1.5.jar JTransforms/3.1//JTransforms-3.1.jar -RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +RoaringBitmap/0.9.0//RoaringBitmap-0.9.0.jar ST4/4.0.4//ST4-4.0.4.jar accessors-smart/1.2//accessors-smart-1.2.jar activation/1.1.1//activation-1.1.1.jar @@ -142,10 +142,10 @@ jodd-core/3.5.2//jodd-core-3.5.2.jar jpam/1.1//jpam-1.1.jar json-smart/2.3//json-smart-2.3.jar json/1.8//json-1.8.jar -json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar -json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +json4s-ast_2.12/3.7.0-M5//json4s-ast_2.12-3.7.0-M5.jar +json4s-core_2.12/3.7.0-M5//json4s-core_2.12-3.7.0-M5.jar +json4s-jackson_2.12/3.7.0-M5//json4s-jackson_2.12-3.7.0-M5.jar +json4s-scalap_2.12/3.7.0-M5//json4s-scalap_2.12-3.7.0-M5.jar jsp-api/2.1//jsp-api-2.1.jar jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar @@ -183,12 +183,12 @@ metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar metrics-json/4.1.1//metrics-json-4.1.1.jar metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.47.Final//netty-all-4.1.47.Final.jar +netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar nimbus-jose-jwt/4.41.1//nimbus-jose-jwt-4.41.1.jar -objenesis/2.5.1//objenesis-2.5.1.jar +objenesis/2.6//objenesis-2.6.jar okhttp/2.7.5//okhttp-2.7.5.jar okhttp/3.12.6//okhttp-3.12.6.jar -okio/1.15.0//okio-1.15.0.jar +okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar orc-core/1.5.10//orc-core-1.5.10.jar orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar @@ -213,7 +213,7 @@ scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar scala-reflect/2.12.10//scala-reflect-2.12.10.jar scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar -shims/0.7.45//shims-0.7.45.jar +shims/0.9.0//shims-0.9.0.jar slf4j-api/1.7.30//slf4j-api-1.7.30.jar slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar snakeyaml/1.24//snakeyaml-1.24.jar diff --git a/dev/github_jira_sync.py b/dev/github_jira_sync.py index b90afeebc5238..d916e2dbd8223 100755 --- a/dev/github_jira_sync.py +++ b/dev/github_jira_sync.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/lint-python b/dev/lint-python index 1fddbfa64b32c..41da41bfda5dd 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -20,7 +20,7 @@ FLAKE8_BUILD="flake8" MINIMUM_FLAKE8="3.5.0" PYCODESTYLE_BUILD="pycodestyle" -MINIMUM_PYCODESTYLE="2.4.0" +MINIMUM_PYCODESTYLE="2.6.0" SPHINX_BUILD="sphinx-build" @@ -173,7 +173,6 @@ function sphinx_test { return fi - # TODO(SPARK-32279): Install Sphinx in Python 3 of Jenkins machines PYTHON_HAS_SPHINX=$("$PYTHON_EXECUTABLE" -c 'import importlib.util; print(importlib.util.find_spec("sphinx") is not None)') if [[ "$PYTHON_HAS_SPHINX" == "False" ]]; then echo "$PYTHON_EXECUTABLE does not have Sphinx installed. Skipping Sphinx build for now." @@ -181,6 +180,23 @@ function sphinx_test { return fi + # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. + # See also https://github.com/sphinx-doc/sphinx/issues/7551. + PYTHON_HAS_SPHINX_3_0=$("$PYTHON_EXECUTABLE" -c 'from distutils.version import LooseVersion; import sphinx; print(LooseVersion(sphinx.__version__) < LooseVersion("3.1.0"))') + if [[ "$PYTHON_HAS_SPHINX_3_0" == "False" ]]; then + echo "$PYTHON_EXECUTABLE has Sphinx 3.1+ installed but it requires lower then 3.1. Skipping Sphinx build for now." + echo + return + fi + + # TODO(SPARK-32391): Install pydata_sphinx_theme in Jenkins machines + PYTHON_HAS_THEME=$("$PYTHON_EXECUTABLE" -c 'import importlib.util; print(importlib.util.find_spec("pydata_sphinx_theme") is not None)') + if [[ "$PYTHON_HAS_THEME" == "False" ]]; then + echo "$PYTHON_EXECUTABLE does not have pydata_sphinx_theme installed. Skipping Sphinx build for now." + echo + return + fi + echo "starting $SPHINX_BUILD tests..." pushd python/docs &> /dev/null make clean &> /dev/null diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index b42429d7175b1..2e376fb343bbe 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/requirements.txt b/dev/requirements.txt index baea9213dbc97..a862a6e986791 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -3,3 +3,4 @@ jira==1.0.3 PyGithub==1.26.0 Unidecode==0.04.19 sphinx +pydata_sphinx_theme diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 5fd0be7476f29..b322d3f61b444 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -63,11 +63,15 @@ fi PYSPARK_VERSION=$(python3 -c "exec(open('python/pyspark/version.py').read());print(__version__)") PYSPARK_DIST="$FWDIR/python/dist/pyspark-$PYSPARK_VERSION.tar.gz" # The pip install options we use for all the pip commands -PIP_OPTIONS="--user --upgrade --no-cache-dir --force-reinstall " +PIP_OPTIONS="--upgrade --no-cache-dir --force-reinstall" # Test both regular user and edit/dev install modes. PIP_COMMANDS=("pip install $PIP_OPTIONS $PYSPARK_DIST" "pip install $PIP_OPTIONS -e python/") +# Jenkins has PySpark installed under user sitepackages shared for some reasons. +# In this test, explicitly exclude user sitepackages to prevent side effects +export PYTHONNOUSERSITE=1 + for python in "${PYTHON_EXECS[@]}"; do for install_command in "${PIP_COMMANDS[@]}"; do echo "Testing pip installation with python $python" @@ -81,7 +85,7 @@ for python in "${PYTHON_EXECS[@]}"; do source "$CONDA_PREFIX/etc/profile.d/conda.sh" fi conda create -y -p "$VIRTUALENV_PATH" python=$python numpy pandas pip setuptools - conda activate "$VIRTUALENV_PATH" || (echo "Falling back to 'source activate'" && source activate "$VIRTUALENV_PATH") + source activate "$VIRTUALENV_PATH" || conda activate "$VIRTUALENV_PATH" else mkdir -p "$VIRTUALENV_PATH" virtualenv --python=$python "$VIRTUALENV_PATH" @@ -96,8 +100,6 @@ for python in "${PYTHON_EXECS[@]}"; do cd "$FWDIR"/python # Delete the egg info file if it exists, this can cache the setup file. rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" - # Also, delete the symbolic link if exists. It can be left over from the previous editable mode installation. - python3 -c "from distutils.sysconfig import get_python_lib; import os; f = os.path.join(get_python_lib(), 'pyspark.egg-link'); os.unlink(f) if os.path.isfile(f) else 0" python3 setup.py sdist @@ -116,7 +118,6 @@ for python in "${PYTHON_EXECS[@]}"; do cd / echo "Run basic sanity check on pip installed version with spark-submit" - export PATH="$(python3 -m site --user-base)/bin:$PATH" spark-submit "$FWDIR"/dev/pip-sanity-check.py echo "Run basic sanity check with import based" python3 "$FWDIR"/dev/pip-sanity-check.py @@ -127,7 +128,7 @@ for python in "${PYTHON_EXECS[@]}"; do # conda / virtualenv environments need to be deactivated differently if [ -n "$USE_CONDA" ]; then - conda deactivate || (echo "Falling back to 'source deactivate'" && source deactivate) + source deactivate || conda deactivate else deactivate fi diff --git a/dev/run-tests.py b/dev/run-tests.py index 065a27c0e853b..6aae3bdaefaff 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -49,13 +49,12 @@ def determine_modules_for_files(filenames): ['pyspark-core', 'sql'] >>> [x.name for x in determine_modules_for_files(["file_not_matched_by_any_subproject"])] ['root'] - >>> [x.name for x in determine_modules_for_files( \ - [".github/workflows/master.yml", "appveyor.yml"])] + >>> [x.name for x in determine_modules_for_files(["appveyor.yml"])] [] """ changed_modules = set() for filename in filenames: - if filename in (".github/workflows/master.yml", "appveyor.yml"): + if filename in ("appveyor.yml",): continue matched_at_least_one_module = False for module in modules.all_modules: @@ -609,11 +608,20 @@ def main(): " install one and retry.") sys.exit(2) - # install SparkR - if which("R"): - run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) - else: - print("Cannot install SparkR as R was not found in PATH") + # Install SparkR + should_only_test_modules = opts.modules is not None + test_modules = [] + if should_only_test_modules: + str_test_modules = [m.strip() for m in opts.modules.split(",")] + test_modules = [m for m in modules.all_modules if m.name in str_test_modules] + + if not should_only_test_modules or modules.sparkr in test_modules: + # If tests modules are specified, we will not run R linter. + # SparkR needs the manual SparkR installation. + if which("R"): + run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) + else: + print("Cannot install SparkR as R was not found in PATH") if os.environ.get("AMPLAB_JENKINS"): # if we're on the Amplab Jenkins build servers setup variables @@ -640,16 +648,11 @@ def main(): "and Hive profile", hive_version, "under environment", test_env) extra_profiles = get_hadoop_profiles(hadoop_version) + get_hive_profiles(hive_version) - changed_modules = None - test_modules = None - changed_files = None - should_only_test_modules = opts.modules is not None + changed_modules = [] + changed_files = [] included_tags = [] excluded_tags = [] if should_only_test_modules: - str_test_modules = [m.strip() for m in opts.modules.split(",")] - test_modules = [m for m in modules.all_modules if m.name in str_test_modules] - # If we're running the tests in Github Actions, attempt to detect and test # only the affected modules. if test_env == "github_actions": diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index b3e68bed1d1e7..129b073d75254 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -29,7 +29,7 @@ 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 -Pkubernetes -Pyarn -Phive" +HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive" MVN="build/mvn" HADOOP_HIVE_PROFILES=( hadoop-2.7-hive-1.2 @@ -79,14 +79,14 @@ for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do HIVE_PROFILE=hive-1.2 fi echo "Performing Maven install for $HADOOP_HIVE_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q + $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q echo "Performing Maven validate for $HADOOP_HIVE_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE validate -q + $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE validate -q echo "Generating dependency manifest for $HADOOP_HIVE_PROFILE" mkdir -p dev/pr-deps - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ + $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ | grep "Dependencies classpath:" -A 1 \ | tail -n 1 | tr ":" "\n" | awk -F '/' '{ # For each dependency classpath, we fetch the last three parts split by "/": artifact id, version, and jar name. diff --git a/dev/tox.ini b/dev/tox.ini index 3ee13c09b5de7..5bf27d1abde9b 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -16,4 +16,4 @@ [pycodestyle] ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 -exclude=cloudpickle.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* +exclude=python/pyspark/cloudpickle/*.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* diff --git a/docs/README.md b/docs/README.md index 22039871cf63d..e2002a66b0433 100644 --- a/docs/README.md +++ b/docs/README.md @@ -57,8 +57,13 @@ Note: Other versions of roxygen2 might work in SparkR documentation generation b To generate API docs for any language, you'll need to install these libraries: + + ```sh -$ sudo pip install sphinx mkdocs numpy +$ sudo pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme ``` ## Generating the Documentation HTML diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index eea657e684495..63f6b4a0a204b 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -187,6 +187,12 @@ url: sql-ref-syntax-qry-select-tvf.html - text: Window Function url: sql-ref-syntax-qry-select-window.html + - text: CASE Clause + url: sql-ref-syntax-qry-select-case.html + - text: LATERAL VIEW Clause + url: sql-ref-syntax-qry-select-lateral-view.html + - text: PIVOT Clause + url: sql-ref-syntax-qry-select-pivot.html - text: EXPLAIN url: sql-ref-syntax-qry-explain.html - text: Auxiliary Statements @@ -208,6 +214,8 @@ url: sql-ref-syntax-aux-cache-clear-cache.html - text: REFRESH TABLE url: sql-ref-syntax-aux-cache-refresh-table.html + - text: REFRESH FUNCTION + url: sql-ref-syntax-aux-cache-refresh-function.html - text: REFRESH url: sql-ref-syntax-aux-cache-refresh.html - text: DESCRIBE @@ -249,6 +257,8 @@ url: sql-ref-syntax-aux-conf-mgmt-set.html - text: RESET url: sql-ref-syntax-aux-conf-mgmt-reset.html + - text: SET TIME ZONE + url: sql-ref-syntax-aux-conf-mgmt-set-timezone.html - text: RESOURCE MANAGEMENT url: sql-ref-syntax-aux-resource-mgmt.html subitems: diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 8e2a06e4bc9a8..17da22bf8a433 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -126,8 +126,8 @@ puts "Making directory api/python" mkdir_p "api/python" - puts "cp -r ../python/docs/_build/html/. api/python" - cp_r("../python/docs/_build/html/.", "api/python") + puts "cp -r ../python/docs/build/html/. api/python" + cp_r("../python/docs/build/html/.", "api/python") end if not (ENV['SKIP_RDOC'] == '1') diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 749d026528017..1eaa8ab4eeee0 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -163,7 +163,7 @@ different stores and connectors when renaming directories: | Amazon S3 | s3a | Unsafe | O(data) | | Azure Storage | wasb | Safe | O(files) | | Azure Datalake Gen 2 | abfs | Safe | O(1) | -| Google GCS | gs | Safe | O(1) | +| Google Cloud Storage | gs | Safe | O(1) | As storing temporary files can run up charges; delete directories called `"_temporary"` on a regular basis. @@ -253,9 +253,9 @@ Here is the documentation on the standard connectors both from Apache and the cl * [Azure Blob Storage and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). * [Azure Data Lake Gen 1](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). * [Hadoop-AWS module (Hadoop 3.x)](https://hadoop.apache.org/docs/current3/hadoop-aws/tools/hadoop-aws/index.html). -* [Amazon S3 via S3A and S3N (Hadoop 2.x)](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). -* [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon -* [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google +* [Amazon S3 via S3A and S3N (Hadoop 2.x)](https://hadoop.apache.org/docs/current2/hadoop-aws/tools/hadoop-aws/index.html). +* [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon. +* [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/dataproc/docs/concepts/connectors/cloud-storage). From Google. * [The Azure Blob Filesystem driver (ABFS)](https://docs.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-abfs-driver) -* IBM Cloud Object Storage connector for Apache Spark: [Stocator](https://github.com/CODAIT/stocator), [IBM Object Storage](https://www.ibm.com/cloud/object-storage). From IBM +* IBM Cloud Object Storage connector for Apache Spark: [Stocator](https://github.com/CODAIT/stocator), [IBM Object Storage](https://www.ibm.com/cloud/object-storage). From IBM. diff --git a/docs/configuration.md b/docs/configuration.md index abf76105ae77d..62799db8becd0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3028,3 +3028,10 @@ There are configurations available to request resources for the driver: sp Spark will use the configurations specified to first request containers with the corresponding resources from the cluster manager. Once it gets the container, Spark launches an Executor in that container which will discover what resources the container has and the addresses associated with each resource. The Executor will register with the Driver and report back the resources available to that Executor. The Spark scheduler can then schedule tasks to each Executor and assign specific resource addresses based on the resource requirements the user specified. The user can see the resources assigned to a task using the TaskContext.get().resources api. On the driver, the user can see the resources assigned with the SparkContext resources call. It's then up to the user to use the assignedaddresses to do the processing they want or pass those into the ML/AI framework they are using. See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with Mesos or local mode. And please also note that local-cluster mode with multiple workers is not supported(see Standalone documentation). + +# Stage Level Scheduling Overview + +The stage level scheduling feature allows users to specify task and executor resource requirements at the stage level. This allows for different stages to run with executors that have different resources. A prime example of this is one ETL stage runs with executors with just CPUs, the next stage is an ML stage that needs GPUs. Stage level scheduling allows for user to request different executors that have GPUs when the ML stage runs rather then having to acquire executors with GPUs at the start of the application and them be idle while the ETL stage is being run. +This is only available for the RDD API in Scala, Java, and Python and requires dynamic allocation to be enabled. It is only available on YARN at this time. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page for more implementation details. + +See the `RDD.withResources` and `ResourceProfileBuilder` API's for using this feature. The current implementation acquires new executors for each `ResourceProfile` created and currently has to be an exact match. Spark does not try to fit tasks into an executor that require a different ResourceProfile than the executor was created with. Executors that are not in use will idle timeout with the dynamic allocation logic. The default configuration for this feature is to only allow one ResourceProfile per stage. If the user associates more then 1 ResourceProfile to an RDD, Spark will throw an exception by default. See config `spark.scheduler.resource.profileMergeConflicts` to control that behavior. The current merge strategy Spark implements when `spark.scheduler.resource.profileMergeConflicts` is enabled is a simple max of each resource within the conflicting ResourceProfiles. Spark will create a new ResourceProfile with the max of each of the resources. diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 63baef145f01e..11d3e0019617f 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -22,6 +22,10 @@ license: | * Table of contents {:toc} +## Upgrading from Core 3.0 to 3.1 + +- In Spark 3.0 and below, `SparkContext` can be created in executors. Since Spark 3.1, an exception will be thrown when creating `SparkContext` in executors. You can allow it by setting the configuration `spark.executor.allowSparkContext` when creating `SparkContext` in executors. + ## Upgrading from Core 2.4 to 3.0 - The `org.apache.spark.ExecutorPlugin` interface and related configuration has been replaced with diff --git a/docs/img/pyspark-components.png b/docs/img/pyspark-components.png new file mode 100644 index 0000000000000..a0979d3465a92 Binary files /dev/null and b/docs/img/pyspark-components.png differ diff --git a/docs/img/pyspark-components.pptx b/docs/img/pyspark-components.pptx new file mode 100644 index 0000000000000..e0111a44e186e Binary files /dev/null and b/docs/img/pyspark-components.pptx differ diff --git a/docs/img/spark-logo-reverse.png b/docs/img/spark-logo-reverse.png new file mode 100644 index 0000000000000..a3e4ed4bb3d08 Binary files /dev/null and b/docs/img/spark-logo-reverse.png differ diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 5c19c77f37a81..1eed0ff3ee5e8 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -297,11 +297,9 @@ via `sc.setJobGroup` in a separate PVM thread, which also disallows to cancel th later. In order to synchronize PVM threads with JVM threads, you should set `PYSPARK_PIN_THREAD` environment variable -to `true`. This pinned thread mode allows one PVM thread has one corresponding JVM thread. - -However, currently it cannot inherit the local properties from the parent thread although it isolates -each thread with its own local properties. To work around this, you should manually copy and set the -local properties from the parent thread to the child thread when you create another thread in PVM. +to `true`. This pinned thread mode allows one PVM thread has one corresponding JVM thread. With this mode, +`pyspark.InheritableThread` is recommanded to use together for a PVM thread to inherit the interitable attributes + such as local properties in a JVM thread. Note that `PYSPARK_PIN_THREAD` is currently experimental and not recommended for use in production. diff --git a/docs/ml-guide.md b/docs/ml-guide.md index ddce98b32f941..1b4a3e4eee557 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -62,23 +62,13 @@ The primary Machine Learning API for Spark is now the [DataFrame](sql-programmin # Dependencies -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing. -If native libraries[^1] are not available at runtime, you will see a warning message and a pure JVM -implementation will be used instead. +MLlib uses linear algebra packages [Breeze](http://www.scalanlp.org/) and [netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing[^1]. Those packages may call native acceleration libraries such as [Intel MKL](https://software.intel.com/content/www/us/en/develop/tools/math-kernel-library.html) or [OpenBLAS](http://www.openblas.net) if they are available as system libraries or in runtime library paths. -Due to licensing issues with runtime proprietary binaries, we do not include `netlib-java`'s native -proxies by default. -To configure `netlib-java` / Breeze to use system optimised binaries, include -`com.github.fommil.netlib:all:1.1.2` (or build Spark with `-Pnetlib-lgpl`) as a dependency of your -project and read the [netlib-java](https://github.com/fommil/netlib-java) documentation for your -platform's additional installation instructions. - -The most popular native BLAS such as [Intel MKL](https://software.intel.com/en-us/mkl), [OpenBLAS](http://www.openblas.net), can use multiple threads in a single operation, which can conflict with Spark's execution model. - -Configuring these BLAS implementations to use a single thread for operations may actually improve performance (see [SPARK-21305](https://issues.apache.org/jira/browse/SPARK-21305)). It is usually optimal to match this to the number of cores each Spark task is configured to use, which is 1 by default and typically left at 1. - -Please refer to resources like the following to understand how to configure the number of threads these BLAS implementations use: [Intel MKL](https://software.intel.com/en-us/articles/recommended-settings-for-calling-intel-mkl-routines-from-multi-threaded-applications) or [Intel oneMKL](https://software.intel.com/en-us/onemkl-linux-developer-guide-improving-performance-with-threading) and [OpenBLAS](https://github.com/xianyi/OpenBLAS/wiki/faq#multi-threaded). Note that if nativeBLAS is not properly configured in system, java implementation(f2jBLAS) will be used as fallback option. +Due to differing OSS licenses, `netlib-java`'s native proxies can't be distributed with Spark. See [MLlib Linear Algebra Acceleration Guide](ml-linalg-guide.html) for how to enable accelerated linear algebra processing. If accelerated native libraries are not enabled, you will see a warning message like below and a pure JVM implementation will be used instead: +``` +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeSystemBLAS +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeRefBLAS +``` To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. diff --git a/docs/ml-linalg-guide.md b/docs/ml-linalg-guide.md new file mode 100644 index 0000000000000..739091363473f --- /dev/null +++ b/docs/ml-linalg-guide.md @@ -0,0 +1,103 @@ +--- +layout: global +title: MLlib Linear Algebra Acceleration Guide +displayTitle: MLlib Linear Algebra Acceleration Guide +license: | + 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. +--- + +## Introduction + +This guide provides necessary information to enable accelerated linear algebra processing for Spark MLlib. + +Spark MLlib defines Vector and Matrix as basic data types for machine learning algorithms. On top of them, [BLAS](https://en.wikipedia.org/wiki/Basic_Linear_Algebra_Subprograms) and [LAPACK](https://en.wikipedia.org/wiki/LAPACK) operations are implemented and supported by [netlib-java](https://github.com/fommil/netlib-Java) (the algorithms may call [Breeze](https://github.com/scalanlp/breeze) and it will in turn call `netlib-java`). `netlib-java` can use optimized native linear algebra libraries (refered to as "native libraries" or "BLAS libraries" hereafter) for faster numerical processing. [Intel MKL](https://software.intel.com/content/www/us/en/develop/tools/math-kernel-library.html) and [OpenBLAS](http://www.openblas.net) are two popular ones. + +However due to license differences, the official released Spark binaries by default don't contain native libraries support for `netlib-java`. + +The following sections describe how to enable `netlib-java` with native libraries support for Spark MLlib and how to install native libraries and configure them properly. + +## Enable `netlib-java` with native library proxies + +`netlib-java` depends on `libgfortran`. It requires GFORTRAN 1.4 or above. This can be obtained by installing `libgfortran` package. After installation, the following command can be used to verify if it is installed properly. +``` +strings /path/to/libgfortran.so.3.0.0 | grep GFORTRAN_1.4 +``` + +To build Spark with `netlib-java` native library proxies, you need to add `-Pnetlib-lgpl` to Maven build command line. For example: +``` +$SPARK_SOURCE_HOME/build/mvn -Pnetlib-lgpl -DskipTests -Pyarn -Phadoop-2.7 clean package +``` + +If you only want to enable it in your project, include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project. + +## Install native linear algebra libraries + +Intel MKL and OpenBLAS are two popular native linear algebra libraries. You can choose one of them based on your preference. We provide basic instructions as below. You can refer to [netlib-java documentation](https://github.com/fommil/netlib-java) for more advanced installation instructions. + +### Intel MKL + +- Download and install Intel MKL. The installation should be done on all nodes of the cluster. We assume the installation location is $MKLROOT (e.g. /opt/intel/mkl). +- Create soft links to `libmkl_rt.so` with specific names in system library search paths. For instance, make sure `/usr/local/lib` is in system library search paths and run the following commands: +``` +$ ln -sf $MKLROOT/lib/intel64/libmkl_rt.so /usr/local/lib/libblas.so.3 +$ ln -sf $MKLROOT/lib/intel64/libmkl_rt.so /usr/local/lib/liblapack.so.3 +``` + +### OpenBLAS + +The installation should be done on all nodes of the cluster. Generic version of OpenBLAS are available with most distributions. You can install it with a distribution package manager like `apt` or `yum`. + +For Debian / Ubuntu: +``` +sudo apt-get install libopenblas-base +sudo update-alternatives --config libblas.so.3 +``` +For CentOS / RHEL: +``` +sudo yum install openblas +``` + +## Check if native libraries are enabled for MLlib + +To verify native libraries are properly loaded, start `spark-shell` and run the following code: +``` +scala> import com.github.fommil.netlib.BLAS; +scala> System.out.println(BLAS.getInstance().getClass().getName()); +``` + +If they are correctly loaded, it should print `com.github.fommil.netlib.NativeSystemBLAS`. Otherwise the warnings should be printed: +``` +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeSystemBLAS +WARN BLAS: Failed to load implementation from:com.github.fommil.netlib.NativeRefBLAS +``` + +If native libraries are not properly configured in the system, the Java implementation (f2jBLAS) will be used as fallback option. + +## Spark Configuration + +The default behavior of multi-threading in either Intel MKL or OpenBLAS may not be optimal with Spark's execution model [^1]. + +Therefore configuring these native libraries to use a single thread for operations may actually improve performance (see [SPARK-21305](https://issues.apache.org/jira/browse/SPARK-21305)). It is usually optimal to match this to the number of `spark.task.cpus`, which is `1` by default and typically left at `1`. + +You can use the options in `config/spark-env.sh` to set thread number for Intel MKL or OpenBLAS: +* For Intel MKL: +``` +MKL_NUM_THREADS=1 +``` +* For OpenBLAS: +``` +OPENBLAS_NUM_THREADS=1 +``` + +[^1]: Please refer to the following resources to understand how to configure the number of threads for these BLAS implementations: [Intel MKL](https://software.intel.com/en-us/articles/recommended-settings-for-calling-intel-mkl-routines-from-multi-threaded-applications) or [Intel oneMKL](https://software.intel.com/en-us/onemkl-linux-developer-guide-improving-performance-with-threading) and [OpenBLAS](https://github.com/xianyi/OpenBLAS/wiki/faq#multi-threaded). diff --git a/docs/monitoring.md b/docs/monitoring.md index 2ab7b30a1dca9..247957d087fa3 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -573,6 +573,7 @@ can be identified by their `[attempt-id]`. In the API listed below, when running ?planDescription=[true (default) | false] enables/disables Physical planDescription on demand when Physical Plan size is high.
?offset=[offset]&length=[len] lists queries in the given range. + /applications/[app-id]/sql/[execution-id] @@ -581,6 +582,7 @@ can be identified by their `[attempt-id]`. In the API listed below, when running ?details=[true (default) | false] lists/hides metric details in addition to given query details.
?planDescription=[true (default) | false] enables/disables Physical planDescription on demand for the given query when Physical Plan size is high. + /applications/[app-id]/environment diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index eeeb71b022c52..6f7aaf2baeccd 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -60,7 +60,7 @@ For example: examples/jars/spark-examples*.jar \ 10 -The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Debugging your Application" section below for how to see driver and executor logs. +The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the [Debugging your Application](#debugging-your-application) section below for how to see driver and executor logs. To launch a Spark application in `client` mode, do the same, but replace `cluster` with `client`. The following shows how you can run `spark-shell` in `client` mode: @@ -96,7 +96,7 @@ in your application jar. To build Spark yourself, refer to [Building Spark](building-spark.html). -To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](running-on-yarn.html#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. +To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. # Configuration @@ -641,6 +641,10 @@ If the user has a user defined YARN resource, lets call it `acceleratorX` then t YARN does not tell Spark the addresses of the resources allocated to each container. For that reason, the user must specify a discovery script that gets run by the executor on startup to discover what resources are available to that executor. You can find an example scripts in `examples/src/main/scripts/getGpusResources.sh`. The script must have execute permissions set and the user should setup permissions to not allow malicious users to modify it. The script should write to STDOUT a JSON string in the format of the ResourceInformation class. This has the resource name and an array of resource addresses available to just that executor. +# Stage Level Scheduling Overview + +Stage level scheduling is supported on YARN when dynamic allocation is enabled. One thing to note that is YARN specific is that each ResourceProfile requires a different container priority on YARN. The mapping is simply the ResourceProfile id becomes the priority, on YARN lower numbers are higher priority. This means that profiles created earlier will have a higher priority in YARN. Normally this won't matter as Spark finishes one stage before starting another one, the only case this might have an affect is in a job server type scenario, so its something to keep in mind. + # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/docs/sparkr.md b/docs/sparkr.md index d86fa86c89853..05310f89f278d 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -674,19 +674,19 @@ Rscript -e 'install.packages("arrow", repos="https://cloud.r-project.org/")' Please refer [the official documentation of Apache Arrow](https://arrow.apache.org/docs/r/) for more detials. Note that you must ensure that Arrow R package is installed and available on all cluster nodes. -The current supported minimum version is 0.15.1; however, this might change between the minor releases since Arrow optimization in SparkR is experimental. +The current supported minimum version is 1.0.0; however, this might change between the minor releases since Arrow optimization in SparkR is experimental. ## Enabling for Conversion to/from R DataFrame, `dapply` and `gapply` Arrow optimization is available when converting a Spark DataFrame to an R DataFrame using the call `collect(spark_df)`, when creating a Spark DataFrame from an R DataFrame with `createDataFrame(r_df)`, when applying an R native function to each partition via `dapply(...)` and when applying an R native function to grouped data via `gapply(...)`. -To use Arrow when executing these calls, users need to first set the Spark configuration ‘spark.sql.execution.arrow.sparkr.enabled’ -to ‘true’. This is disabled by default. +To use Arrow when executing these, users need to set the Spark configuration ‘spark.sql.execution.arrow.sparkr.enabled’ +to ‘true’ first. This is disabled by default. -In addition, optimizations enabled by ‘spark.sql.execution.arrow.sparkr.enabled’ could fallback automatically to non-Arrow optimization -implementation if an error occurs before the actual computation within Spark during converting a Spark DataFrame to/from an R -DataFrame. +Whether the optimization is enabled or not, SparkR produces the same results. In addition, the conversion +between Spark DataFrame and R DataFrame falls back automatically to non-Arrow optimization implementation +when the optimization fails for any reasons before the actual computation.
{% highlight r %} @@ -713,9 +713,9 @@ collect(gapply(spark_df, {% endhighlight %}
-Using the above optimizations with Arrow will produce the same results as when Arrow is not enabled. Note that even with Arrow, -`collect(spark_df)` results in the collection of all records in the DataFrame to the driver program and should be done on a -small subset of the data. +Note that even with Arrow, `collect(spark_df)` results in the collection of all records in the DataFrame to +the driver program and should be done on a small subset of the data. In addition, the specified output schema +in `gapply(...)` and `dapply(...)` should be matched to the R DataFrame's returned by the given function. ## Supported SQL Types diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index d3138ae319160..c7f6116b88f87 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -24,11 +24,17 @@ license: | ## Upgrading from Spark SQL 3.0 to 3.1 - - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. + - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.integerGroupingId` to `true`. - - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. + - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.1, you can set `spark.sql.ui.explainMode` to `extended`. - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. + + - In Spark 3.1, the Parquet, ORC, Avro and JSON datasources throw the exception `org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema` in read if they detect duplicate names in top-level columns as well in nested structures. The datasources take into account the SQL config `spark.sql.caseSensitive` while detecting column name duplicates. + + - In Spark 3.1, structs and maps are wrapped by the `{}` brackets in casting them to strings. For instance, the `show()` action and the `CAST` expression use such brackets. In Spark 3.0 and earlier, the `[]` brackets are used for the same purpose. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`. + + - In Spark 3.1, NULL elements of structures, arrays and maps are converted to "null" in casting them to strings. In Spark 3.0 or earlier, NULL elements are converted to empty strings. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`. ## Upgrading from Spark SQL 3.0 to 3.0.1 @@ -148,7 +154,8 @@ license: | - In Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and so on. Spark 3.0 uses Java 8 API classes from the `java.time` packages that are based on [ISO chronology](https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and below, those operations are performed using the hybrid calendar ([Julian + Gregorian](https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: - * Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in `sql-ref-datetime-pattern.md`, which is implemented via `java.time.format.DateTimeFormatter` under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and below, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in [simpleDateFormat](https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html). The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`. + * Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in [Datetime Patterns for Formatting and Parsing](sql-ref-datetime-pattern.html), + which is implemented via [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and below, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in [SimpleDateFormat](https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html). The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`. * The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 6488ad9cd34c9..e786c0bc9aff8 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -264,7 +264,7 @@ Below is a list of all the keywords in Spark SQL. |MAP|non-reserved|non-reserved|non-reserved| |MATCHED|non-reserved|non-reserved|non-reserved| |MERGE|non-reserved|non-reserved|non-reserved| -|MINUS|not-reserved|strict-non-reserved|non-reserved| +|MINUS|non-reserved|strict-non-reserved|non-reserved| |MINUTE|reserved|non-reserved|reserved| |MONTH|reserved|non-reserved|reserved| |MSCK|non-reserved|non-reserved|non-reserved| @@ -355,6 +355,7 @@ Below is a list of all the keywords in Spark SQL. |TEMPORARY|non-reserved|non-reserved|non-reserved| |TERMINATED|non-reserved|non-reserved|non-reserved| |THEN|reserved|non-reserved|reserved| +|TIME|reserved|non-reserved|reserved| |TO|reserved|non-reserved|reserved| |TOUCH|non-reserved|non-reserved|non-reserved| |TRAILING|reserved|non-reserved|reserved| @@ -385,3 +386,4 @@ Below is a list of all the keywords in Spark SQL. |WINDOW|non-reserved|non-reserved|reserved| |WITH|reserved|non-reserved|reserved| |YEAR|reserved|non-reserved|reserved| +|ZONE|non-reserved|non-reserved|non-reserved| diff --git a/docs/sql-ref-datetime-pattern.md b/docs/sql-ref-datetime-pattern.md index 3c0bc754f940b..d0299e5a99dc2 100644 --- a/docs/sql-ref-datetime-pattern.md +++ b/docs/sql-ref-datetime-pattern.md @@ -70,7 +70,7 @@ The count of pattern letters determines the format. For formatting, the fraction length would be padded to the number of contiguous 'S' with zeros. Spark supports datetime of micro-of-second precision, which has up to 6 significant digits, but can parse nano-of-second with exceeded part truncated. -- Year: The count of letters determines the minimum field width below which padding is used. If the count of letters is two, then a reduced two digit form is used. For printing, this outputs the rightmost two digits. For parsing, this will parse using the base value of 2000, resulting in a year within the range 2000 to 2099 inclusive. If the count of letters is less than four (but not two), then the sign is only output for negative years. Otherwise, the sign is output if the pad width is exceeded when 'G' is not present. 11 or more letters will fail. +- Year: The count of letters determines the minimum field width below which padding is used. If the count of letters is two, then a reduced two digit form is used. For printing, this outputs the rightmost two digits. For parsing, this will parse using the base value of 2000, resulting in a year within the range 2000 to 2099 inclusive. If the count of letters is less than four (but not two), then the sign is only output for negative years. Otherwise, the sign is output if the pad width is exceeded when 'G' is not present. 7 or more letters will fail. - Month: It follows the rule of Number/Text. The text form is depend on letters - 'M' denotes the 'standard' form, and 'L' is for 'stand-alone' form. These two forms are different only in some certain languages. For example, in Russian, 'Июль' is the stand-alone form of July, and 'Июля' is the standard form. Here are examples for all supported pattern letters: - `'M'` or `'L'`: Month number in a year starting from 1. There is no difference between 'M' and 'L'. Month from 1 to 9 are printed without padding. diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md index fdef3d657dfa3..8829016fc17ac 100644 --- a/docs/sql-ref-syntax-aux-cache-cache-table.md +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -80,3 +80,4 @@ CACHE TABLE testCache OPTIONS ('storageLevel' 'DISK_ONLY') SELECT * FROM testDat * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md index a27cd83c146a3..aae4e39600375 100644 --- a/docs/sql-ref-syntax-aux-cache-clear-cache.md +++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md @@ -41,3 +41,4 @@ CLEAR CACHE; * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache-refresh-function.md b/docs/sql-ref-syntax-aux-cache-refresh-function.md new file mode 100644 index 0000000000000..d91fc062eb714 --- /dev/null +++ b/docs/sql-ref-syntax-aux-cache-refresh-function.md @@ -0,0 +1,60 @@ +--- +layout: global +title: REFRESH FUNCTION +displayTitle: REFRESH FUNCTION +license: | + 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. +--- + +### Description + +`REFRESH FUNCTION` statement invalidates the cached function entry, which includes a class name +and resource location of the given function. The invalidated cache is populated right away. +Note that `REFRESH FUNCTION` only works for permanent functions. Refreshing native functions or temporary functions will cause an exception. + +### Syntax + +```sql +REFRESH FUNCTION function_identifier +``` + +### Parameters + +* **function_identifier** + + Specifies a function name, which is either a qualified or unqualified name. If no database identifier is provided, uses the current database. + + **Syntax:** `[ database_name. ] function_name` + +### Examples + +```sql +-- The cached entry of the function will be refreshed +-- The function is resolved from the current database as the function name is unqualified. +REFRESH FUNCTION func1; + +-- The cached entry of the function will be refreshed +-- The function is resolved from tempDB database as the function name is qualified. +REFRESH FUNCTION db1.func1; +``` + +### Related Statements + +* [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) +* [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) +* [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) +* [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) +* [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-cache-refresh-table.md b/docs/sql-ref-syntax-aux-cache-refresh-table.md index 8d4a804f88671..cc35c0451d5cb 100644 --- a/docs/sql-ref-syntax-aux-cache-refresh-table.md +++ b/docs/sql-ref-syntax-aux-cache-refresh-table.md @@ -57,3 +57,4 @@ REFRESH TABLE tempDB.view1; * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache-refresh.md b/docs/sql-ref-syntax-aux-cache-refresh.md index b10e6fb47aaf7..715bdcac3b6f1 100644 --- a/docs/sql-ref-syntax-aux-cache-refresh.md +++ b/docs/sql-ref-syntax-aux-cache-refresh.md @@ -54,3 +54,4 @@ REFRESH "hdfs://path/to/table"; * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index 96a691e4c3931..4456378cdee15 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -50,3 +50,4 @@ UNCACHE TABLE t1; * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) +* [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md index 0ccb1c61a0da5..17a13e67e532e 100644 --- a/docs/sql-ref-syntax-aux-cache.md +++ b/docs/sql-ref-syntax-aux-cache.md @@ -23,4 +23,5 @@ license: | * [UNCACHE TABLE statement](sql-ref-syntax-aux-cache-uncache-table.html) * [CLEAR CACHE statement](sql-ref-syntax-aux-cache-clear-cache.html) * [REFRESH TABLE statement](sql-ref-syntax-aux-cache-refresh-table.html) - * [REFRESH statement](sql-ref-syntax-aux-cache-refresh.html) \ No newline at end of file + * [REFRESH statement](sql-ref-syntax-aux-cache-refresh.html) + * [REFRESH FUNCTION statement](sql-ref-syntax-aux-cache-refresh-function.html) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index 4caf57a232f89..68df0913b29d4 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -21,19 +21,34 @@ license: | ### Description -Reset any runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. +The RESET command resets runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. ### Syntax ```sql -RESET +RESET; + +RESET configuration_key; ``` +### Parameters + +* **(none)** + + Reset any runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. + +* **configuration_key** + + Restore the value of the `configuration_key` to the default value. If the default value is undefined, the `configuration_key` will be removed. + ### Examples ```sql -- Reset any runtime configurations specific to the current session which were set via the SET command to their default values. RESET; + +-- If you start your application with --conf spark.foo=bar and set spark.foo=foobar in runtime, the example below will restore it to 'bar'. If spark.foo is not specified during starting, the example bellow will remove this config from the SQLConf. It will ignore nonexistent keys. +RESET spark.abc; ``` ### Related Statements diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-set-timezone.md b/docs/sql-ref-syntax-aux-conf-mgmt-set-timezone.md new file mode 100644 index 0000000000000..47dd2be77ae90 --- /dev/null +++ b/docs/sql-ref-syntax-aux-conf-mgmt-set-timezone.md @@ -0,0 +1,67 @@ +--- +layout: global +title: SET TIME ZONE +displayTitle: SET TIME ZONE +license: | + 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. +--- + +### Description + +The SET TIME ZONE command sets the time zone of the current session. + +### Syntax + +```sql +SET TIME ZONE LOCAL +SET TIME ZONE 'timezone_value' +SET TIME ZONE INTERVAL interval_literal +``` + +### Parameters + +* **LOCAL** + + Set the time zone to the one specified in the java `user.timezone` property, or to the environment variable `TZ` if `user.timezone` is undefined, or to the system time zone if both of them are undefined. + +* **timezone_value** + + The ID of session local timezone in the format of either region-based zone IDs or zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in the format '`(+|-)HH`', '`(+|-)HH:mm`' or '`(+|-)HH:mm:ss`', e.g '-08', '+01:00' or '-13:33:33'. Also, 'UTC' and 'Z' are supported as aliases of '+00:00'. Other short names are not recommended to use because they can be ambiguous. + +* **interval_literal** + + The [interval literal](sql-ref-literals.html#interval-literal) represents the difference between the session time zone to the 'UTC'. It must be in the range of [-18, 18] hours and max to second precision, e.g. `INTERVAL 2 HOURS 30 MINITUES` or `INTERVAL '15:40:32' HOUR TO SECOND`. + +### Examples + +```sql +-- Set time zone to the system default. +SET TIME ZONE LOCAL; + +-- Set time zone to the region-based zone ID. +SET TIME ZONE 'America/Los_Angeles'; + +-- Set time zone to the Zone offset. +SET TIME ZONE '+08:00'; + +-- Set time zone with intervals. +SET TIME ZONE INTERVAL 1 HOUR 30 MINUTES; +SET TIME ZONE INTERVAL '08:30:00' HOUR TO SECOND; +``` + +### Related Statements + +* [SET](sql-ref-syntax-aux-conf-mgmt-set.html) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt.md b/docs/sql-ref-syntax-aux-conf-mgmt.md index 1900fb7f1cb9a..3312bcb503500 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt.md @@ -21,3 +21,4 @@ license: | * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) * [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) + * [SET TIME ZONE](sql-ref-syntax-aux-conf-mgmt-set-timezone.html) diff --git a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md index 38f8856a24e3d..7bf847df98150 100644 --- a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md +++ b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md @@ -36,6 +36,14 @@ CREATE [ EXTERNAL ] TABLE [ IF NOT EXISTS ] table_identifier [ LOCATION path ] [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] [ AS select_statement ] + +row_format: + : SERDE serde_class [ WITH SERDEPROPERTIES (k1=v1, k2=v2, ... ) ] + | DELIMITED [ FIELDS TERMINATED BY fields_termiated_char [ ESCAPED BY escaped_char ] ] + [ COLLECTION ITEMS TERMINATED BY collection_items_termiated_char ] + [ MAP KEYS TERMINATED BY map_key_termiated_char ] + [ LINES TERMINATED BY row_termiated_char ] + [ NULL DEFINED AS null_char ] ``` Note that, the clauses between the columns definition clause and the AS SELECT clause can come in @@ -51,15 +59,55 @@ as any order. For example, you can write COMMENT table_comment after TBLPROPERTI * **EXTERNAL** - Table is defined using the path provided as LOCATION, does not use default location for this table. + Table is defined using the path provided as `LOCATION`, does not use default location for this table. * **PARTITIONED BY** Partitions are created on the table, based on the columns specified. + +* **row_format** + + Use the `SERDE` clause to specify a custom SerDe for one table. Otherwise, use the `DELIMITED` clause to use the native SerDe and specify the delimiter, escape character, null character and so on. + +* **SERDE** + + Specifies a custom SerDe for one table. + +* **serde_class** + + Specifies a fully-qualified class name of a custom SerDe. + +* **SERDEPROPERTIES** + + A list of key-value pairs that is used to tag the SerDe definition. + +* **DELIMITED** + + The `DELIMITED` clause can be used to specify the native SerDe and state the delimiter, escape character, null character and so on. + +* **FIELDS TERMINATED BY** -* **ROW FORMAT** + Used to define a column separator. + +* **COLLECTION ITEMS TERMINATED BY** - SERDE is used to specify a custom SerDe or the DELIMITED clause in order to use the native SerDe. + Used to define a collection item separator. + +* **MAP KEYS TERMINATED BY** + + Used to define a map key separator. + +* **LINES TERMINATED BY** + + Used to define a row separator. + +* **NULL DEFINED AS** + + Used to define the specific value for NULL. + +* **ESCAPED BY** + + Used for escape mechanism. * **STORED AS** @@ -114,9 +162,47 @@ CREATE TABLE student (id INT, name STRING) PARTITIONED BY (age INT); --Use Row Format and file format -CREATE TABLE student (id INT,name STRING) +CREATE TABLE student (id INT, name STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE; + +--Use complex datatype +CREATE EXTERNAL TABLE family( + name STRING, + friends ARRAY, + children MAP, + address STRUCT + ) + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' ESCAPED BY '\\' + COLLECTION ITEMS TERMINATED BY '_' + MAP KEYS TERMINATED BY ':' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'foonull' + STORED AS TEXTFILE + LOCATION '/tmp/family/'; + +--Use predefined custom SerDe +CREATE TABLE avroExample + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + TBLPROPERTIES ('avro.schema.literal'='{ "namespace": "org.apache.hive", + "name": "first_schema", + "type": "record", + "fields": [ + { "name":"string1", "type":"string" }, + { "name":"string2", "type":"string" } + ] }'); + +--Use personalized custom SerDe(we may need to `ADD JAR xxx.jar` first to ensure we can find the serde_class, +--or you may run into `CLASSNOTFOUND` exception) +ADD JAR /tmp/hive_serde_example.jar; + +CREATE EXTERNAL TABLE family (id INT, name STRING) + ROW FORMAT SERDE 'com.ly.spark.serde.SerDeExample' + STORED AS INPUTFORMAT 'com.ly.spark.example.serde.io.SerDeExampleInputFormat' + OUTPUTFORMAT 'com.ly.spark.example.serde.io.SerDeExampleOutputFormat' + LOCATION '/tmp/family/'; ``` ### Related Statements diff --git a/docs/sql-ref-syntax-qry-select-case.md b/docs/sql-ref-syntax-qry-select-case.md new file mode 100644 index 0000000000000..6136b161d234b --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-case.md @@ -0,0 +1,109 @@ +--- +layout: global +title: CASE Clause +displayTitle: CASE Clause +license: | + 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. +--- + +### Description + +`CASE` clause uses a rule to return a specific result based on the specified condition, similar to if/else statements in other programming languages. + +### Syntax + +```sql +CASE [ expression ] { WHEN boolean_expression THEN then_expression } [ ... ] + [ ELSE else_expression ] +END +``` + +### Parameters + +* **boolean_expression** + + Specifies any expression that evaluates to a result type `boolean`. Two or + more expressions may be combined together using the logical + operators ( `AND`, `OR` ). + +* **then_expression** + + Specifies the then expression based on the `boolean_expression` condition; `then_expression` and `else_expression` should all be same type or coercible to a common type. + +* **else_expression** + + Specifies the default expression; `then_expression` and `else_expression` should all be same type or coercible to a common type. + +### Examples + +```sql +CREATE TABLE person (id INT, name STRING, age INT); +INSERT INTO person VALUES + (100, 'John', 30), + (200, 'Mary', NULL), + (300, 'Mike', 80), + (400, 'Dan', 50); + +SELECT id, CASE WHEN id > 200 THEN 'bigger' ELSE 'small' END FROM person; ++------+--------------------------------------------------+ +| id | CASE WHEN (id > 200) THEN bigger ELSE small END | ++------+--------------------------------------------------+ +| 100 | small | +| 200 | small | +| 300 | bigger | +| 400 | bigger | ++------+--------------------------------------------------+ + +SELECT id, CASE id WHEN 100 then 'bigger' WHEN id > 300 THEN '300' ELSE 'small' END FROM person; ++------+-----------------------------------------------------------------------------------------------+ +| id | CASE WHEN (id = 100) THEN bigger WHEN (id = CAST((id > 300) AS INT)) THEN 300 ELSE small END | ++------+-----------------------------------------------------------------------------------------------+ +| 100 | bigger | +| 200 | small | +| 300 | small | +| 400 | small | ++------+-----------------------------------------------------------------------------------------------+ + +SELECT * FROM person + WHERE + CASE 1 = 1 + WHEN 100 THEN 'big' + WHEN 200 THEN 'bigger' + WHEN 300 THEN 'biggest' + ELSE 'small' + END = 'small'; ++------+-------+-------+ +| id | name | age | ++------+-------+-------+ +| 100 | John | 30 | +| 200 | Mary | NULL | +| 300 | Mike | 80 | +| 400 | Dan | 50 | ++------+-------+-------+ +``` + +### Related Statements + +* [SELECT Main](sql-ref-syntax-qry-select.html) +* [WHERE Clause](sql-ref-syntax-qry-select-where.html) +* [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +* [HAVING Clause](sql-ref-syntax-qry-select-having.html) +* [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +* [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +* [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-clusterby.md b/docs/sql-ref-syntax-qry-select-clusterby.md index e3bd2ed926ecc..9bcfac5b3b5c6 100644 --- a/docs/sql-ref-syntax-qry-select-clusterby.md +++ b/docs/sql-ref-syntax-qry-select-clusterby.md @@ -99,3 +99,6 @@ SELECT age, name FROM person CLUSTER BY age; * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-distribute-by.md b/docs/sql-ref-syntax-qry-select-distribute-by.md index 1fdfb91dad286..fbf662d3359f4 100644 --- a/docs/sql-ref-syntax-qry-select-distribute-by.md +++ b/docs/sql-ref-syntax-qry-select-distribute-by.md @@ -94,3 +94,6 @@ SELECT age, name FROM person DISTRIBUTE BY age; * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md index bd9377ef78df6..6137c0d80f313 100644 --- a/docs/sql-ref-syntax-qry-select-groupby.md +++ b/docs/sql-ref-syntax-qry-select-groupby.md @@ -260,6 +260,30 @@ SELECT city, car_model, sum(quantity) AS sum FROM dealer | San Jose| HondaAccord| 8| | San Jose| HondaCivic| 5| +---------+------------+---+ + +--Prepare data for ignore nulls example +CREATE TABLE person (id INT, name STRING, age INT); +INSERT INTO person VALUES + (100, 'Mary', NULL), + (200, 'John', 30), + (300, 'Mike', 80), + (400, 'Dan', 50); + +--Select the first row in cloumn age +SELECT FIRST(age) FROM person; ++--------------------+ +| first(age, false) | ++--------------------+ +| NULL | ++--------------------+ + +--Get the first row in cloumn `age` ignore nulls,last row in column `id` and sum of cloumn `id`. +SELECT FIRST(age IGNORE NULLS), LAST(id), SUM(id) FROM person; ++-------------------+------------------+----------+ +| first(age, true) | last(id, false) | sum(id) | ++-------------------+------------------+----------+ +| 30 | 400 | 1000 | ++-------------------+------------------+----------+ ``` ### Related Statements @@ -272,3 +296,6 @@ SELECT city, car_model, sum(quantity) AS sum FROM dealer * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md index 935782c551e1f..59a8c680945e6 100644 --- a/docs/sql-ref-syntax-qry-select-having.md +++ b/docs/sql-ref-syntax-qry-select-having.md @@ -125,3 +125,6 @@ SELECT sum(quantity) AS sum FROM dealer HAVING sum(quantity) > 10; * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-lateral-view.md b/docs/sql-ref-syntax-qry-select-lateral-view.md new file mode 100644 index 0000000000000..f742c8fa57043 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-lateral-view.md @@ -0,0 +1,125 @@ +--- +layout: global +title: LATERAL VIEW Clause +displayTitle: LATERAL VIEW Clause +license: | + 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. +--- + +### Description + +The `LATERAL VIEW` clause is used in conjunction with generator functions such as `EXPLODE`, which will generate a virtual table containing one or more rows. `LATERAL VIEW` will apply the rows to each original output row. + +### Syntax + +```sql +LATERAL VIEW [ OUTER ] generator_function ( expression [ , ... ] ) [ table_alias ] AS column_alias [ , ... ] +``` + +### Parameters + +* **OUTER** + + If `OUTER` specified, returns null if an input array/map is empty or null. + +* **generator_function** + + Specifies a generator function (EXPLODE, INLINE, etc.). + +* **table_alias** + + The alias for `generator_function`, which is optional. + +* **column_alias** + + Lists the column aliases of `generator_function`, which may be used in output rows. We may have multiple aliases if `generator_function` have multiple output columns. + +### Examples + +```sql +CREATE TABLE person (id INT, name STRING, age INT, class INT, address STRING); +INSERT INTO person VALUES + (100, 'John', 30, 1, 'Street 1'), + (200, 'Mary', NULL, 1, 'Street 2'), + (300, 'Mike', 80, 3, 'Street 3'), + (400, 'Dan', 50, 4, 'Street 4'); + +SELECT * FROM person + LATERAL VIEW EXPLODE(ARRAY(30, 60)) tabelName AS c_age + LATERAL VIEW EXPLODE(ARRAY(40, 80)) AS d_age; ++------+-------+-------+--------+-----------+--------+--------+ +| id | name | age | class | address | c_age | d_age | ++------+-------+-------+--------+-----------+--------+--------+ +| 100 | John | 30 | 1 | Street 1 | 30 | 40 | +| 100 | John | 30 | 1 | Street 1 | 30 | 80 | +| 100 | John | 30 | 1 | Street 1 | 60 | 40 | +| 100 | John | 30 | 1 | Street 1 | 60 | 80 | +| 200 | Mary | NULL | 1 | Street 2 | 30 | 40 | +| 200 | Mary | NULL | 1 | Street 2 | 30 | 80 | +| 200 | Mary | NULL | 1 | Street 2 | 60 | 40 | +| 200 | Mary | NULL | 1 | Street 2 | 60 | 80 | +| 300 | Mike | 80 | 3 | Street 3 | 30 | 40 | +| 300 | Mike | 80 | 3 | Street 3 | 30 | 80 | +| 300 | Mike | 80 | 3 | Street 3 | 60 | 40 | +| 300 | Mike | 80 | 3 | Street 3 | 60 | 80 | +| 400 | Dan | 50 | 4 | Street 4 | 30 | 40 | +| 400 | Dan | 50 | 4 | Street 4 | 30 | 80 | +| 400 | Dan | 50 | 4 | Street 4 | 60 | 40 | +| 400 | Dan | 50 | 4 | Street 4 | 60 | 80 | ++------+-------+-------+--------+-----------+--------+--------+ + +SELECT c_age, COUNT(1) FROM person + LATERAL VIEW EXPLODE(ARRAY(30, 60)) AS c_age + LATERAL VIEW EXPLODE(ARRAY(40, 80)) AS d_age +GROUP BY c_age; ++--------+-----------+ +| c_age | count(1) | ++--------+-----------+ +| 60 | 8 | +| 30 | 8 | ++--------+-----------+ + +SELECT * FROM person + LATERAL VIEW EXPLODE(ARRAY()) tabelName AS c_age; ++-----+-------+------+--------+----------+--------+ +| id | name | age | class | address | c_age | ++-----+-------+------+--------+----------+--------+ ++-----+-------+------+--------+----------+--------+ + +SELECT * FROM person + LATERAL VIEW OUTER EXPLODE(ARRAY()) tabelName AS c_age; ++------+-------+-------+--------+-----------+--------+ +| id | name | age | class | address | c_age | ++------+-------+-------+--------+-----------+--------+ +| 100 | John | 30 | 1 | Street 1 | NULL | +| 200 | Mary | NULL | 1 | Street 2 | NULL | +| 300 | Mike | 80 | 3 | Street 3 | NULL | +| 400 | Dan | 50 | 4 | Street 4 | NULL | ++------+-------+-------+--------+-----------+--------+ +``` + +### Related Statements + +* [SELECT Main](sql-ref-syntax-qry-select.html) +* [WHERE Clause](sql-ref-syntax-qry-select-where.html) +* [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +* [HAVING Clause](sql-ref-syntax-qry-select-having.html) +* [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +* [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +* [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) diff --git a/docs/sql-ref-syntax-qry-select-limit.md b/docs/sql-ref-syntax-qry-select-limit.md index 03c4df3cbc442..bd64ba890901b 100644 --- a/docs/sql-ref-syntax-qry-select-limit.md +++ b/docs/sql-ref-syntax-qry-select-limit.md @@ -104,3 +104,6 @@ org.apache.spark.sql.AnalysisException: The limit expression must evaluate to a * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-orderby.md b/docs/sql-ref-syntax-qry-select-orderby.md index 85bbe514cdc95..13f0ae40cb828 100644 --- a/docs/sql-ref-syntax-qry-select-orderby.md +++ b/docs/sql-ref-syntax-qry-select-orderby.md @@ -143,3 +143,6 @@ SELECT * FROM person ORDER BY name ASC, age DESC; * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-pivot.md b/docs/sql-ref-syntax-qry-select-pivot.md new file mode 100644 index 0000000000000..649c2518b28eb --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-pivot.md @@ -0,0 +1,101 @@ +--- +layout: global +title: PIVOT Clause +displayTitle: PIVOT Clause +license: | + 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. +--- + +### Description + +The `PIVOT` clause is used for data perspective. We can get the aggregated values based on specific column values, which will be turned to multiple columns used in `SELECT` clause. The `PIVOT` clause can be specified after the table name or subquery. + +### Syntax + +```sql +PIVOT ( { aggregate_expression [ AS aggregate_expression_alias ] } [ , ... ] + FOR column_list IN ( expression_list ) ) +``` + +### Parameters + +* **aggregate_expression** + + Specifies an aggregate expression (SUM(a), COUNT(DISTINCT b), etc.). + +* **aggregate_expression_alias** + + Specifies an alias for the aggregate expression. + +* **column_list** + + Contains columns in the `FROM` clause, which specifies the columns we want to replace with new columns. We can use brackets to surround the columns, such as `(c1, c2)`. + +* **expression_list** + + Specifies new columns, which are used to match values in `column_list` as the aggregating condition. We can also add aliases for them. + +### Examples + +```sql +CREATE TABLE person (id INT, name STRING, age INT, class INT, address STRING); +INSERT INTO person VALUES + (100, 'John', 30, 1, 'Street 1'), + (200, 'Mary', NULL, 1, 'Street 2'), + (300, 'Mike', 80, 3, 'Street 3'), + (400, 'Dan', 50, 4, 'Street 4'); + +SELECT * FROM person + PIVOT ( + SUM(age) AS a, AVG(class) AS c + FOR name IN ('John' AS john, 'Mike' AS mike) + ); ++------+-----------+---------+---------+---------+---------+ +| id | address | john_a | john_c | mike_a | mike_c | ++------+-----------+---------+---------+---------+---------+ +| 200 | Street 2 | NULL | NULL | NULL | NULL | +| 100 | Street 1 | 30 | 1.0 | NULL | NULL | +| 300 | Street 3 | NULL | NULL | 80 | 3.0 | +| 400 | Street 4 | NULL | NULL | NULL | NULL | ++------+-----------+---------+---------+---------+---------+ + +SELECT * FROM person + PIVOT ( + SUM(age) AS a, AVG(class) AS c + FOR (name, age) IN (('John', 30) AS c1, ('Mike', 40) AS c2) + ); ++------+-----------+-------+-------+-------+-------+ +| id | address | c1_a | c1_c | c2_a | c2_c | ++------+-----------+-------+-------+-------+-------+ +| 200 | Street 2 | NULL | NULL | NULL | NULL | +| 100 | Street 1 | 30 | 1.0 | NULL | NULL | +| 300 | Street 3 | NULL | NULL | NULL | NULL | +| 400 | Street 4 | NULL | NULL | NULL | NULL | ++------+-----------+-------+-------+-------+-------+ +``` + +### Related Statements + +* [SELECT Main](sql-ref-syntax-qry-select.html) +* [WHERE Clause](sql-ref-syntax-qry-select-where.html) +* [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +* [HAVING Clause](sql-ref-syntax-qry-select-having.html) +* [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +* [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +* [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-sortby.md b/docs/sql-ref-syntax-qry-select-sortby.md index 554bdb569d005..09e559adcd6ab 100644 --- a/docs/sql-ref-syntax-qry-select-sortby.md +++ b/docs/sql-ref-syntax-qry-select-sortby.md @@ -176,3 +176,6 @@ SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select-where.md b/docs/sql-ref-syntax-qry-select-where.md index ca3f5ec7866c6..9ff7993d40c58 100644 --- a/docs/sql-ref-syntax-qry-select-where.md +++ b/docs/sql-ref-syntax-qry-select-where.md @@ -125,3 +125,6 @@ SELECT * FROM person AS parent * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md index 987e6479ab20a..453737a2f36b8 100644 --- a/docs/sql-ref-syntax-qry-select.md +++ b/docs/sql-ref-syntax-qry-select.md @@ -31,10 +31,10 @@ of a query along with examples. ```sql [ WITH with_query [ , ... ] ] select_statement [ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select_statement, ... ] - [ ORDER BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] - [ SORT BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] - [ CLUSTER BY { expression [ , ...] } ] - [ DISTRIBUTE BY { expression [, ...] } ] + [ ORDER BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ... ] } ] + [ SORT BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ... ] } ] + [ CLUSTER BY { expression [ , ... ] } ] + [ DISTRIBUTE BY { expression [, ... ] } ] [ WINDOW { named_window [ , WINDOW named_window, ... ] } ] [ LIMIT { ALL | expression } ] ``` @@ -42,9 +42,11 @@ select_statement [ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select_stat While `select_statement` is defined as ```sql SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } - FROM { from_item [ , ...] } + FROM { from_item [ , ... ] } + [ PIVOT clause ] + [ LATERAL VIEW clause ] [ ... ] [ WHERE boolean_expression ] - [ GROUP BY expression [ , ...] ] + [ GROUP BY expression [ , ... ] ] [ HAVING boolean_expression ] ``` @@ -75,7 +77,7 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } **Syntax:** `expression [AS] [alias]` - * **from_item** +* **from_item** Specifies a source of input for the query. It can be one of the following: * Table relation @@ -83,62 +85,71 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } * [Table-value function](sql-ref-syntax-qry-select-tvf.html) * [Inline table](sql-ref-syntax-qry-select-inline-table.html) * Subquery + +* **PIVOT** + The `PIVOT` clause is used for data perspective; We can get the aggregated values based on specific column value. - * **WHERE** +* **LATERAL VIEW** + + The `LATERAL VIEW` clause is used in conjunction with generator functions such as `EXPLODE`, which will generate a virtual table containing one or more rows. `LATERAL VIEW` will apply the rows to each original output row. + +* **WHERE** Filters the result of the FROM clause based on the supplied predicates. - * **GROUP BY** +* **GROUP BY** Specifies the expressions that are used to group the rows. This is used in conjunction with aggregate functions (MIN, MAX, COUNT, SUM, AVG, etc.) to group rows based on the grouping expressions and aggregate values in each group. When a FILTER clause is attached to an aggregate function, only the matching rows are passed to that function. - * **HAVING** +* **HAVING** Specifies the predicates by which the rows produced by GROUP BY are filtered. The HAVING clause is used to filter rows after the grouping is performed. If HAVING is specified without GROUP BY, it indicates a GROUP BY without grouping expressions (global aggregate). - * **ORDER BY** +* **ORDER BY** Specifies an ordering of the rows of the complete result set of the query. The output rows are ordered across the partitions. This parameter is mutually exclusive with `SORT BY`, `CLUSTER BY` and `DISTRIBUTE BY` and can not be specified together. - * **SORT BY** +* **SORT BY** Specifies an ordering by which the rows are ordered within each partition. This parameter is mutually exclusive with `ORDER BY` and `CLUSTER BY` and can not be specified together. - * **CLUSTER BY** +* **CLUSTER BY** Specifies a set of expressions that is used to repartition and sort the rows. Using this clause has the same effect of using `DISTRIBUTE BY` and `SORT BY` together. - * **DISTRIBUTE BY** +* **DISTRIBUTE BY** Specifies a set of expressions by which the result rows are repartitioned. This parameter is mutually exclusive with `ORDER BY` and `CLUSTER BY` and can not be specified together. - * **LIMIT** +* **LIMIT** Specifies the maximum number of rows that can be returned by a statement or subquery. This clause is mostly used in the conjunction with `ORDER BY` to produce a deterministic result. - * **boolean_expression** +* **boolean_expression** - Specifies an expression with a return type of boolean. + Specifies any expression that evaluates to a result type `boolean`. Two or + more expressions may be combined together using the logical + operators ( `AND`, `OR` ). - * **expression** +* **expression** Specifies a combination of one or more values, operators, and SQL functions that evaluates to a value. - * **named_window** +* **named_window** - Specifies aliases for one or more source window specifications. The source window specifications can - be referenced in the widow definitions in the query. + Specifies aliases for one or more source window specifications. The source window specifications can + be referenced in the widow definitions in the query. ### Related Statements @@ -159,3 +170,6 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } * [TABLESAMPLE](sql-ref-syntax-qry-select-sampling.html) * [Table-valued Function](sql-ref-syntax-qry-select-tvf.html) * [Window Function](sql-ref-syntax-qry-select-window.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry.md b/docs/sql-ref-syntax-qry.md index 167c394d0fe49..d55ea43d15036 100644 --- a/docs/sql-ref-syntax-qry.md +++ b/docs/sql-ref-syntax-qry.md @@ -45,4 +45,7 @@ ability to generate logical and physical plan for a given query using * [TABLESAMPLE](sql-ref-syntax-qry-select-sampling.html) * [Table-valued Function](sql-ref-syntax-qry-select-tvf.html) * [Window Function](sql-ref-syntax-qry-select-window.html) + * [CASE Clause](sql-ref-syntax-qry-select-case.html) + * [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) + * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) * [EXPLAIN Statement](sql-ref-syntax-qry-explain.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index 4bf1858428d98..4e58abb2a8596 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -66,6 +66,9 @@ Spark SQL is Apache Spark's module for working with structured data. The SQL Syn * [Table-valued Function](sql-ref-syntax-qry-select-tvf.html) * [WHERE Clause](sql-ref-syntax-qry-select-where.html) * [Window Function](sql-ref-syntax-qry-select-window.html) + * [CASE Clause](sql-ref-syntax-qry-select-case.html) + * [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) + * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) * [EXPLAIN](sql-ref-syntax-qry-explain.html) ### Auxiliary Statements @@ -83,6 +86,7 @@ Spark SQL is Apache Spark's module for working with structured data. The SQL Syn * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) * [REFRESH TABLE](sql-ref-syntax-aux-cache-refresh-table.html) + * [REFRESH FUNCTION](sql-ref-syntax-aux-cache-refresh-function.html) * [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) * [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala index c6312d71cc912..a606cc4218778 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala @@ -59,13 +59,13 @@ object BinaryClassificationMetricsExample { // Precision by threshold val precision = metrics.precisionByThreshold - precision.foreach { case (t, p) => + precision.collect.foreach { case (t, p) => println(s"Threshold: $t, Precision: $p") } // Recall by threshold val recall = metrics.recallByThreshold - recall.foreach { case (t, r) => + recall.collect.foreach { case (t, r) => println(s"Threshold: $t, Recall: $r") } @@ -74,13 +74,13 @@ object BinaryClassificationMetricsExample { // F-measure val f1Score = metrics.fMeasureByThreshold - f1Score.foreach { case (t, f) => + f1Score.collect.foreach { case (t, f) => println(s"Threshold: $t, F-score: $f, Beta = 1") } val beta = 0.5 val fScore = metrics.fMeasureByThreshold(beta) - f1Score.foreach { case (t, f) => + fScore.collect.foreach { case (t, f) => println(s"Threshold: $t, F-score: $f, Beta = 0.5") } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala index 5e400b7d715b4..6ed59a3366d93 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala @@ -53,7 +53,7 @@ object ChiSqSelectorExample { // $example off$ println("filtered data: ") - filteredData.foreach(x => println(x)) + filteredData.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala index 1855058bb4f3c..d6ec678b422f9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala @@ -45,10 +45,10 @@ object ElementwiseProductExample { // $example off$ println("transformedData: ") - transformedData.foreach(x => println(x)) + transformedData.collect.foreach(x => println(x)) println("transformedData2: ") - transformedData2.foreach(x => println(x)) + transformedData2.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala index b3a9604c2be3e..b1cad7bf8125f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala @@ -46,10 +46,10 @@ object NormalizerExample { // $example off$ println("data1: ") - data1.foreach(x => println(x)) + data1.collect.foreach(x => println(x)) println("data2: ") - data2.foreach(x => println(x)) + data2.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala index 769fc17b3dc65..66a608ced0f83 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala @@ -49,10 +49,10 @@ object StandardScalerExample { // $example off$ println("data1: ") - data1.foreach(x => println(x)) + data1.collect.foreach(x => println(x)) println("data2: ") - data2.foreach(x => println(x)) + data2.collect.foreach(x => println(x)) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala index a5bdcd8f2ed32..14b2a2015c241 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala @@ -55,10 +55,10 @@ object TFIDFExample { // $example off$ println("tfidf: ") - tfidf.foreach(x => println(x)) + tfidf.collect.foreach(x => println(x)) println("tfidfIgnore: ") - tfidfIgnore.foreach(x => println(x)) + tfidfIgnore.collect.foreach(x => println(x)) sc.stop() } diff --git a/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt index 3c1b5af0d5986..b70b1446f6f64 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt @@ -2,121 +2,129 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2689 2694 7 5.8 170.9 1.0X +Sum 2872 2936 90 5.5 182.6 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2741 2759 26 5.7 174.2 1.0X +Sum 2810 2838 40 5.6 178.6 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2736 2748 17 5.7 173.9 1.0X +Sum 2901 2922 30 5.4 184.4 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3305 3317 17 4.8 210.2 1.0X +Sum 3387 3391 5 4.6 215.4 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2904 2952 68 5.4 184.6 1.0X +Sum 2890 2960 99 5.4 183.7 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3090 3093 4 5.1 196.5 1.0X +Sum 3067 3088 30 5.1 195.0 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 5351 5365 20 2.0 510.3 1.0X +Sum of columns 4736 4818 116 2.2 451.7 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 3278 3288 14 4.8 208.4 1.0X -Partition column 3149 3193 62 5.0 200.2 1.0X -Both columns 3198 3204 7 4.9 203.4 1.0X +Data column 3383 3400 23 4.6 215.1 1.0X +Partition column 2949 2959 14 5.3 187.5 1.1X +Both columns 3522 3545 33 4.5 223.9 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3435 3438 5 3.1 327.6 1.0X +Sum of string length 3332 3355 32 3.1 317.7 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 5634 5650 23 1.9 537.3 1.0X +Sum of string length 5588 5652 90 1.9 532.9 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 4725 4752 39 2.2 450.6 1.0X +Sum of string length 3858 3865 9 2.7 368.0 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3550 3566 23 3.0 338.6 1.0X +Sum of string length 2562 2571 12 4.1 244.3 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 5271 5279 11 0.2 5027.0 1.0X +Sum of single column 5241 5243 3 0.2 4998.0 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 10393 10516 174 0.1 9911.3 1.0X +Sum of single column 10178 10185 10 0.1 9706.5 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 15330 15343 19 0.1 14619.6 1.0X +Sum of single column 15201 15232 44 0.1 14496.4 1.0X +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 9614 9669 54 0.1 9614.1 1.0X +pushdown disabled 10077 10141 66 0.1 10077.2 1.0X +w/ filters 4681 4713 29 0.2 4681.5 2.1X + diff --git a/external/avro/benchmarks/AvroReadBenchmark-results.txt b/external/avro/benchmarks/AvroReadBenchmark-results.txt index 0ab611a0f9a29..3108a9c8e13fe 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-results.txt @@ -2,121 +2,129 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3049 3071 32 5.2 193.8 1.0X +Sum 2841 2846 7 5.5 180.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2982 2992 13 5.3 189.6 1.0X +Sum 2777 2799 30 5.7 176.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2984 2989 7 5.3 189.7 1.0X +Sum 2730 2753 33 5.8 173.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3262 3353 128 4.8 207.4 1.0X +Sum 3278 3284 9 4.8 208.4 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2716 2723 10 5.8 172.7 1.0X +Sum 2801 2805 6 5.6 178.1 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2868 2870 3 5.5 182.4 1.0X +Sum 2976 2984 12 5.3 189.2 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 4714 4739 35 2.2 449.6 1.0X +Sum of columns 4674 4686 17 2.2 445.8 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 3257 3286 41 4.8 207.1 1.0X -Partition column 3258 3277 27 4.8 207.2 1.0X -Both columns 3399 3405 9 4.6 216.1 1.0X +Data column 3273 3284 17 4.8 208.1 1.0X +Partition column 2934 2935 2 5.4 186.6 1.1X +Both columns 3395 3405 14 4.6 215.8 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3292 3316 33 3.2 314.0 1.0X +Sum of string length 3340 3353 19 3.1 318.5 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 5450 5456 9 1.9 519.7 1.0X +Sum of string length 5484 5493 12 1.9 523.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 4410 4435 35 2.4 420.6 1.0X +Sum of string length 3817 3833 22 2.7 364.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3074 3122 68 3.4 293.2 1.0X +Sum of string length 2340 2354 20 4.5 223.2 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 5120 5136 23 0.2 4882.7 1.0X +Sum of single column 4709 4719 14 0.2 4491.1 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 9952 10002 71 0.1 9490.7 1.0X +Sum of single column 9159 9171 18 0.1 8734.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 14973 14978 7 0.1 14279.8 1.0X +Sum of single column 13645 13751 151 0.1 13012.8 1.0X +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 9215 9309 146 0.1 9215.2 1.0X +pushdown disabled 9535 9637 96 0.1 9534.9 1.0X +w/ filters 3969 3994 22 0.3 3969.5 2.3X + diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 79c72057c5823..285a30bcd046e 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -98,7 +98,10 @@ case class AvroDataToCatalyst( try { decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) result = reader.read(result, decoder) - deserializer.deserialize(result) + val deserialized = deserializer.deserialize(result) + assert(deserialized.isDefined, + "Avro deserializer cannot return an empty result because filters are not pushed down") + deserialized.get } catch { // There could be multiple possible exceptions here, e.g. java.io.IOException, // AvroRuntimeException, ArrayIndexOutOfBoundsException, etc. diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index 1d18594fd349c..29385b78e3490 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -30,7 +30,7 @@ import org.apache.avro.Schema.Type._ import org.apache.avro.generic._ import org.apache.avro.util.Utf8 -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, StructFilters} import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY @@ -45,12 +45,15 @@ import org.apache.spark.unsafe.types.UTF8String class AvroDeserializer( rootAvroType: Schema, rootCatalystType: DataType, - datetimeRebaseMode: LegacyBehaviorPolicy.Value) { + datetimeRebaseMode: LegacyBehaviorPolicy.Value, + filters: StructFilters) { def this(rootAvroType: Schema, rootCatalystType: DataType) { - this(rootAvroType, rootCatalystType, - LegacyBehaviorPolicy.withName( - SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ))) + this( + rootAvroType, + rootCatalystType, + LegacyBehaviorPolicy.withName(SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ)), + new NoopFilters) } private lazy val decimalConversions = new DecimalConversion() @@ -61,19 +64,20 @@ class AvroDeserializer( private val timestampRebaseFunc = DataSourceUtils.creteTimestampRebaseFuncInRead( datetimeRebaseMode, "Avro") - private val converter: Any => Any = rootCatalystType match { + private val converter: Any => Option[Any] = rootCatalystType match { // A shortcut for empty schema. case st: StructType if st.isEmpty => - (data: Any) => InternalRow.empty + (data: Any) => Some(InternalRow.empty) case st: StructType => val resultRow = new SpecificInternalRow(st.map(_.dataType)) val fieldUpdater = new RowUpdater(resultRow) - val writer = getRecordWriter(rootAvroType, st, Nil) + val applyFilters = filters.skipRow(resultRow, _) + val writer = getRecordWriter(rootAvroType, st, Nil, applyFilters) (data: Any) => { val record = data.asInstanceOf[GenericRecord] - writer(fieldUpdater, record) - resultRow + val skipRow = writer(fieldUpdater, record) + if (skipRow) None else Some(resultRow) } case _ => @@ -82,11 +86,11 @@ class AvroDeserializer( val writer = newWriter(rootAvroType, rootCatalystType, Nil) (data: Any) => { writer(fieldUpdater, 0, data) - tmpRow.get(0, rootCatalystType) + Some(tmpRow.get(0, rootCatalystType)) } } - def deserialize(data: Any): Any = converter(data) + def deserialize(data: Any): Option[Any] = converter(data) /** * Creates a writer to write avro values to Catalyst values at the given ordinal with the given @@ -178,7 +182,9 @@ class AvroDeserializer( updater.setDecimal(ordinal, decimal) case (RECORD, st: StructType) => - val writeRecord = getRecordWriter(avroType, st, path) + // Avro datasource doesn't accept filters with nested attributes. See SPARK-32328. + // We can always return `false` from `applyFilters` for nested records. + val writeRecord = getRecordWriter(avroType, st, path, applyFilters = _ => false) (updater, ordinal, value) => val row = new SpecificInternalRow(st) writeRecord(new RowUpdater(row), value.asInstanceOf[GenericRecord]) @@ -315,7 +321,8 @@ class AvroDeserializer( private def getRecordWriter( avroType: Schema, sqlType: StructType, - path: List[String]): (CatalystDataUpdater, GenericRecord) => Unit = { + path: List[String], + applyFilters: Int => Boolean): (CatalystDataUpdater, GenericRecord) => Boolean = { val validFieldIndexes = ArrayBuffer.empty[Int] val fieldWriters = ArrayBuffer.empty[(CatalystDataUpdater, Any) => Unit] @@ -350,10 +357,13 @@ class AvroDeserializer( (fieldUpdater, record) => { var i = 0 - while (i < validFieldIndexes.length) { + var skipRow = false + while (i < validFieldIndexes.length && !skipRow) { fieldWriters(i)(fieldUpdater, record.get(validFieldIndexes(i))) + skipRow = applyFilters(i) i += 1 } + skipRow } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 59d54bc433f8b..fa4b6b829bdde 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters} import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, Filter} @@ -122,38 +122,28 @@ private[sql] class AvroFileFormat extends FileFormat } reader.sync(file.start) - val stop = file.start + file.length val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( reader.asInstanceOf[DataFileReader[_]].getMetaString, SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ)) - val deserializer = new AvroDeserializer( - userProvidedSchema.getOrElse(reader.getSchema), requiredSchema, datetimeRebaseMode) - - new Iterator[InternalRow] { - private[this] var completed = false - - override def hasNext: Boolean = { - if (completed) { - false - } else { - val r = reader.hasNext && !reader.pastSync(stop) - if (!r) { - reader.close() - completed = true - } - r - } - } + val avroFilters = if (SQLConf.get.avroFilterPushDown) { + new OrderedFilters(filters, requiredSchema) + } else { + new NoopFilters + } - override def next(): InternalRow = { - if (!hasNext) { - throw new NoSuchElementException("next on empty iterator") - } - val record = reader.next() - deserializer.deserialize(record).asInstanceOf[InternalRow] - } + new Iterator[InternalRow] with AvroUtils.RowReader { + override val fileReader = reader + override val deserializer = new AvroDeserializer( + userProvidedSchema.getOrElse(reader.getSchema), + requiredSchema, + datetimeRebaseMode, + avroFilters) + override val stopPosition = file.start + file.length + + override def hasNext: Boolean = hasNextRow + override def next(): InternalRow = nextRow } } else { Iterator.empty diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala index 70dcd58a600fc..51cc51e70cd18 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.avro import java.io.{FileNotFoundException, IOException} import org.apache.avro.Schema +import org.apache.avro.file.{DataFileReader, FileReader} import org.apache.avro.file.DataFileConstants.{BZIP2_CODEC, DEFLATE_CODEC, SNAPPY_CODEC, XZ_CODEC} -import org.apache.avro.file.DataFileReader import org.apache.avro.generic.{GenericDatumReader, GenericRecord} import org.apache.avro.mapred.{AvroOutputFormat, FsInput} import org.apache.avro.mapreduce.AvroJob @@ -32,6 +32,7 @@ import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.AvroOptions.ignoreExtensionKey +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.OutputWriterFactory import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -161,4 +162,37 @@ object AvroUtils extends Logging { "No Avro files found. If files don't have .avro extension, set ignoreExtension to true") } } + + // The trait provides iterator-like interface for reading records from an Avro file, + // deserializing and returning them as internal rows. + trait RowReader { + protected val fileReader: FileReader[GenericRecord] + protected val deserializer: AvroDeserializer + protected val stopPosition: Long + + private[this] var completed = false + private[this] var currentRow: Option[InternalRow] = None + + def hasNextRow: Boolean = { + do { + val r = fileReader.hasNext && !fileReader.pastSync(stopPosition) + if (!r) { + fileReader.close() + completed = true + currentRow = None + } else { + val record = fileReader.next() + currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]] + } + } while (!completed && currentRow.isEmpty) + + currentRow.isDefined + } + + def nextRow: InternalRow = { + currentRow.getOrElse { + throw new NoSuchElementException("next on empty iterator") + } + } + } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala index 15918f46a83bb..1e6c382041efb 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -29,12 +29,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, AvroUtils} +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters} import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile} import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionReaderWithPartitionValues} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -54,7 +55,8 @@ case class AvroPartitionReaderFactory( dataSchema: StructType, readDataSchema: StructType, partitionSchema: StructType, - parsedOptions: AvroOptions) extends FilePartitionReaderFactory with Logging { + parsedOptions: AvroOptions, + filters: Seq[Filter]) extends FilePartitionReaderFactory with Logging { override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { val conf = broadcastedConf.value.value @@ -86,38 +88,28 @@ case class AvroPartitionReaderFactory( } reader.sync(partitionedFile.start) - val stop = partitionedFile.start + partitionedFile.length val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( reader.asInstanceOf[DataFileReader[_]].getMetaString, SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ)) - val deserializer = new AvroDeserializer( - userProvidedSchema.getOrElse(reader.getSchema), readDataSchema, datetimeRebaseMode) - val fileReader = new PartitionReader[InternalRow] { - private[this] var completed = false - - override def next(): Boolean = { - if (completed) { - false - } else { - val r = reader.hasNext && !reader.pastSync(stop) - if (!r) { - reader.close() - completed = true - } - r - } - } + val avroFilters = if (SQLConf.get.avroFilterPushDown) { + new OrderedFilters(filters, readDataSchema) + } else { + new NoopFilters + } - override def get(): InternalRow = { - if (!next) { - throw new NoSuchElementException("next on empty iterator") - } - val record = reader.next() - deserializer.deserialize(record).asInstanceOf[InternalRow] - } + val fileReader = new PartitionReader[InternalRow] with AvroUtils.RowReader { + override val fileReader = reader + override val deserializer = new AvroDeserializer( + userProvidedSchema.getOrElse(reader.getSchema), + readDataSchema, + datetimeRebaseMode, + avroFilters) + override val stopPosition = partitionedFile.start + partitionedFile.length + override def next(): Boolean = hasNextRow + override def get(): InternalRow = nextRow override def close(): Unit = reader.close() } new PartitionReaderWithPartitionValues(fileReader, readDataSchema, diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala index fe7315c739296..e94bef2f8bebe 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -37,6 +38,7 @@ case class AvroScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter], partitionFilters: Seq[Expression] = Seq.empty, dataFilters: Seq[Expression] = Seq.empty) extends FileScan { override def isSplitable(path: Path): Boolean = true @@ -50,8 +52,14 @@ case class AvroScan( val parsedOptions = new AvroOptions(caseSensitiveMap, hadoopConf) // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. - AvroPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + AvroPartitionReaderFactory( + sparkSession.sessionState.conf, + broadcastedConf, + dataSchema, + readDataSchema, + readPartitionSchema, + parsedOptions, + pushedFilters) } override def withFilters( @@ -59,10 +67,18 @@ case class AvroScan( this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { - case a: AvroScan => super.equals(a) && dataSchema == a.dataSchema && options == a.options - + case a: AvroScan => super.equals(a) && dataSchema == a.dataSchema && options == a.options && + equivalentFilters(pushedFilters, a.pushedFilters) case _ => false } override def hashCode(): Int = super.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") + } + + override def getMetaData(): Map[String, String] = { + super.getMetaData() ++ Map("PushedFilers" -> seqToString(pushedFilters)) + } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala index e36c71ef4b1f7..9420608bb22ce 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.v2.avro import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.catalyst.StructFilters +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -29,8 +31,27 @@ class AvroScanBuilder ( schema: StructType, dataSchema: StructType, options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { + override def build(): Scan = { - AvroScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + AvroScan( + sparkSession, + fileIndex, + dataSchema, + readDataSchema(), + readPartitionSchema(), + options, + pushedFilters()) } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (sparkSession.sessionState.conf.avroFilterPushDown) { + _pushedFilters = StructFilters.pushedFilters(filters, dataSchema) + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index c8a1f670bda9e..2d3209f8daa26 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -26,11 +26,14 @@ import org.apache.avro.message.{BinaryMessageDecoder, BinaryMessageEncoder} import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.{RandomDataGenerator, Row} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, NoopFilters, OrderedFilters, StructFilters} import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.{EqualTo, Not} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String class AvroCatalystDataConversionSuite extends SparkFunSuite with SharedSparkSession @@ -272,6 +275,25 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite assert(message == "Cannot convert Catalyst type StringType to Avro type \"long\".") } + private def checkDeserialization( + schema: Schema, + data: GenericData.Record, + expected: Option[Any], + filters: StructFilters = new NoopFilters): Unit = { + val dataType = SchemaConverters.toSqlType(schema).dataType + val deserializer = new AvroDeserializer( + schema, + dataType, + SQLConf.LegacyBehaviorPolicy.CORRECTED, + filters) + val deserialized = deserializer.deserialize(data) + expected match { + case None => assert(deserialized == None) + case Some(d) => + assert(checkResult(d, deserialized.get, dataType, exprNullable = false)) + } + } + test("avro array can be generic java collection") { val jsonFormatSchema = """ @@ -287,30 +309,53 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite |} """.stripMargin val avroSchema = new Schema.Parser().parse(jsonFormatSchema) - val dataType = SchemaConverters.toSqlType(avroSchema).dataType - val deserializer = new AvroDeserializer(avroSchema, dataType) - - def checkDeserialization(data: GenericData.Record, expected: Any): Unit = { - assert(checkResult( - expected, - deserializer.deserialize(data), - dataType, exprNullable = false - )) - } def validateDeserialization(array: java.util.Collection[Integer]): Unit = { val data = new GenericRecordBuilder(avroSchema) .set("array", array) .build() val expected = InternalRow(new GenericArrayData(new util.ArrayList[Any](array))) - checkDeserialization(data, expected) + checkDeserialization(avroSchema, data, Some(expected)) val reEncoded = new BinaryMessageDecoder[GenericData.Record](new GenericData(), avroSchema) .decode(new BinaryMessageEncoder(new GenericData(), avroSchema).encode(data)) - checkDeserialization(reEncoded, expected) + checkDeserialization(avroSchema, reEncoded, Some(expected)) } validateDeserialization(Collections.emptySet()) validateDeserialization(util.Arrays.asList(1, null, 3)) } + + test("SPARK-32346: filter pushdown to Avro deserializer") { + val schema = + """ + |{ + | "type" : "record", + | "name" : "test_schema", + | "fields" : [ + | {"name": "Age", "type": "int"}, + | {"name": "Name", "type": "string"} + | ] + |} + """.stripMargin + val avroSchema = new Schema.Parser().parse(schema) + val sqlSchema = new StructType().add("Age", "int").add("Name", "string") + val data = new GenericRecordBuilder(avroSchema) + .set("Age", 39) + .set("Name", "Maxim") + .build() + val expectedRow = Some(InternalRow(39, UTF8String.fromString("Maxim"))) + + checkDeserialization(avroSchema, data, expectedRow) + checkDeserialization( + avroSchema, + data, + expectedRow, + new OrderedFilters(Seq(EqualTo("Age", 39)), sqlSchema)) + checkDeserialization( + avroSchema, + data, + None, + new OrderedFilters(Seq(Not(EqualTo("Age", 39))), sqlSchema)) + } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 83a7ef0061fb2..8a8a7681abd1c 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -50,9 +50,10 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.v2.avro.AvroScan import org.apache.spark.util.Utils -abstract class AvroSuite extends QueryTest with SharedSparkSession { +abstract class AvroSuite extends QueryTest with SharedSparkSession with NestedDataSourceSuiteBase { import testImplicits._ + override val nestedDataSources = Seq("avro") val episodesAvro = testFile("episodes.avro") val testAvro = testFile("test.avro") @@ -1920,6 +1921,7 @@ class AvroV2Suite extends AvroSuite with ExplainSuiteHelper { |Format: avro |Location: InMemoryFileIndex\\[.*\\] |PartitionFilters: \\[isnotnull\\(id#x\\), \\(id#x > 1\\)\\] + |PushedFilers: \\[IsNotNull\\(value\\), GreaterThan\\(value,2\\)\\] |ReadSchema: struct\\ |""".stripMargin.trim spark.range(10) @@ -1933,7 +1935,38 @@ class AvroV2Suite extends AvroSuite with ExplainSuiteHelper { .format("avro") .load(basePath).where($"id" > 1 && $"value" > 2) val normalizedOutput = getNormalizedExplain(df, FormattedMode) - assert(expected_plan_fragment.r.findAllMatchIn(normalizedOutput).length == 1) + assert(expected_plan_fragment.r.findAllMatchIn(normalizedOutput).length == 1, + normalizedOutput) + } + } + + test("SPARK-32346: filters pushdown to Avro datasource v2") { + Seq(true, false).foreach { filtersPushdown => + withSQLConf(SQLConf.AVRO_FILTER_PUSHDOWN_ENABLED.key -> filtersPushdown.toString) { + withTempPath { dir => + Seq(("a", 1, 2), ("b", 1, 2), ("c", 2, 1)) + .toDF("value", "p1", "p2") + .write + .format("avro") + .save(dir.getCanonicalPath) + val df = spark + .read + .format("avro") + .load(dir.getCanonicalPath) + .where("value = 'a'") + + val fileScan = df.queryExecution.executedPlan collectFirst { + case BatchScanExec(_, f: AvroScan) => f + } + assert(fileScan.nonEmpty) + if (filtersPushdown) { + assert(fileScan.get.pushedFilters.nonEmpty) + } else { + assert(fileScan.get.pushedFilters.isEmpty) + } + checkAnswer(df, Row("a", 1, 2)) + } + } } } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala index dc9606f405191..fde858e0a7419 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.execution.benchmark import java.io.File +import java.time.Instant import scala.util.Random import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -36,6 +39,8 @@ import org.apache.spark.sql.types._ * }}} */ object AvroReadBenchmark extends SqlBasedBenchmark { + import spark.implicits._ + def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) } @@ -186,6 +191,60 @@ object AvroReadBenchmark extends SqlBasedBenchmark { } } + private def filtersPushdownBenchmark(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("Filters pushdown", rowsNum, output = output) + val colsNum = 100 + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", TimestampType)) + val schema = StructType(StructField("key", LongType) +: fields) + def columns(): Seq[Column] = { + val ts = Seq.tabulate(colsNum) { i => + lit(Instant.ofEpochSecond(-30610224000L + i * 123456)).as(s"col$i") + } + ($"id" % 1000).as("key") +: ts + } + withTempPath { path => + // Write and read timestamp in the LEGACY mode to make timestamp conversions more expensive + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> "LEGACY") { + spark.range(rowsNum).select(columns(): _*) + .write + .format("avro") + .save(path.getAbsolutePath) + } + def readback = { + spark.read + .schema(schema) + .format("avro") + .load(path.getAbsolutePath) + } + + benchmark.addCase("w/o filters", numIters) { _ => + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") { + readback.noop() + } + } + + def withFilter(configEnabled: Boolean): Unit = { + withSQLConf( + SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY", + SQLConf.AVRO_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) { + readback.filter($"key" === 0).noop() + } + } + + benchmark.addCase("pushdown disabled", numIters) { _ => + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") { + withFilter(configEnabled = false) + } + } + + benchmark.addCase("w/ filters", numIters) { _ => + withFilter(configEnabled = true) + } + + benchmark.run() + } + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("SQL Single Numeric Column Scan") { Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { dataType => @@ -211,5 +270,8 @@ object AvroReadBenchmark extends SqlBasedBenchmark { columnsBenchmark(1024 * 1024 * 1, 200) columnsBenchmark(1024 * 1024 * 1, 300) } + // Benchmark pushdown filters that refer to top-level columns. + // TODO (SPARK-32328): Add benchmarks for filters with nested column attributes. + filtersPushdownBenchmark(rowsNum = 1000 * 1000, numIters = 3) } } diff --git a/external/docker/spark-test/base/Dockerfile b/external/docker/spark-test/base/Dockerfile index 5bec5d3f16548..d4a30c4681cba 100644 --- a/external/docker/spark-test/base/Dockerfile +++ b/external/docker/spark-test/base/Dockerfile @@ -15,23 +15,14 @@ # limitations under the License. # -FROM ubuntu:xenial +FROM ubuntu:20.04 # Upgrade package index -# install a few other useful packages plus Open Jdk 8 +# install a few other useful packages plus Open Java 11 # Remove unneeded /var/lib/apt/lists/* after install to reduce the # docker image size (by ~30MB) RUN apt-get update && \ - apt-get install -y less openjdk-8-jre-headless iproute2 vim-tiny sudo openssh-server && \ + apt-get install -y less openjdk-11-jre-headless iproute2 vim-tiny sudo openssh-server && \ rm -rf /var/lib/apt/lists/* -ENV SCALA_VERSION 2.12.10 -ENV CDH_VERSION cdh4 -ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark -ENV PATH $SPARK_HOME:$SCALA_HOME/bin:$PATH - -# Install Scala -ADD https://www.scala-lang.org/files/archive/scala-$SCALA_VERSION.tgz / -RUN (cd / && gunzip < scala-$SCALA_VERSION.tgz)|(cd /opt && tar -xvf -) -RUN rm /scala-$SCALA_VERSION.tgz diff --git a/external/docker/spark-test/master/default_cmd b/external/docker/spark-test/master/default_cmd index 5a7da3446f6d2..96a36cd0bb682 100755 --- a/external/docker/spark-test/master/default_cmd +++ b/external/docker/spark-test/master/default_cmd @@ -22,7 +22,4 @@ echo "CONTAINER_IP=$IP" export SPARK_LOCAL_IP=$IP export SPARK_PUBLIC_DNS=$IP -# Avoid the default Docker behavior of mapping our IP address to an unreachable host name -umount /etc/hosts - /opt/spark/bin/spark-class org.apache.spark.deploy.master.Master -i $IP diff --git a/external/docker/spark-test/worker/default_cmd b/external/docker/spark-test/worker/default_cmd index 31b06cb0eb047..2401f5565aa0b 100755 --- a/external/docker/spark-test/worker/default_cmd +++ b/external/docker/spark-test/worker/default_cmd @@ -22,7 +22,4 @@ echo "CONTAINER_IP=$IP" export SPARK_LOCAL_IP=$IP export SPARK_PUBLIC_DNS=$IP -# Avoid the default Docker behavior of mapping our IP address to an unreachable host name -umount /etc/hosts - /opt/spark/bin/spark-class org.apache.spark.deploy.worker.Worker $1 diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 5ab7862674956..6d30bd2a6d2cd 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -18,12 +18,9 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.concurrent.Executors import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration import scala.util.control.NonFatal import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetAndTimestamp} @@ -33,7 +30,7 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} +import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner} /** * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka. @@ -51,19 +48,13 @@ private[kafka010] class KafkaOffsetReader( val driverKafkaParams: ju.Map[String, Object], readerOptions: CaseInsensitiveMap[String], driverGroupIdPrefix: String) extends Logging { + /** - * Used to ensure execute fetch operations execute in an UninterruptibleThread + * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an + * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an + * [[UninterruptibleThread]], however for batch mode this is not the case. */ - val kafkaReaderThread = Executors.newSingleThreadExecutor((r: Runnable) => { - val t = new UninterruptibleThread("Kafka Offset Reader") { - override def run(): Unit = { - r.run() - } - } - t.setDaemon(true) - t - }) - val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread) + val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader") /** * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is @@ -126,14 +117,14 @@ private[kafka010] class KafkaOffsetReader( * Closes the connection to Kafka, and cleans up state. */ def close(): Unit = { - if (_consumer != null) runUninterruptibly { stopConsumer() } - kafkaReaderThread.shutdown() + if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() } + uninterruptibleThreadRunner.shutdown() } /** * @return The Set of TopicPartitions for a given topic */ - def fetchTopicPartitions(): Set[TopicPartition] = runUninterruptibly { + def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly { assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) // Poll to get the latest assigned partitions consumer.poll(0) @@ -531,7 +522,7 @@ private[kafka010] class KafkaOffsetReader( private def partitionsAssignedToConsumer( body: ju.Set[TopicPartition] => Map[TopicPartition, Long], fetchingEarliestOffset: Boolean = false) - : Map[TopicPartition, Long] = runUninterruptibly { + : Map[TopicPartition, Long] = uninterruptibleThreadRunner.runUninterruptibly { withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions @@ -551,23 +542,6 @@ private[kafka010] class KafkaOffsetReader( } } - /** - * This method ensures that the closure is called in an [[UninterruptibleThread]]. - * This is required when communicating with the [[KafkaConsumer]]. In the case - * of streaming queries, we are already running in an [[UninterruptibleThread]], - * however for batch mode this is not the case. - */ - private def runUninterruptibly[T](body: => T): T = { - if (!Thread.currentThread.isInstanceOf[UninterruptibleThread]) { - val future = Future { - body - }(execContext) - ThreadUtils.awaitResult(future, Duration.Inf) - } else { - body - } - } - /** * Helper function that does multiple retries on a body of code that returns offsets. * Retries are needed to handle transient failures. For e.g. race conditions between getting diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index 76c25980fc33f..14dcbeef0d9a3 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -193,6 +193,7 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") + .option("kafka.request.timeout.ms", "3000") .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index a1c7b5ae13474..54ce1717acc71 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -223,6 +223,7 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") + .option("kafka.request.timeout.ms", "3000") .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", "failOnDataLoss.*") .option("startingOffsets", "earliest") 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 ee31652eaf1f4..63659989dec1b 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 @@ -363,6 +363,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") + .option("kafka.request.timeout.ms", "3000") .option("kafka.default.api.timeout.ms", "3000") .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") @@ -400,6 +401,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") + .option("kafka.request.timeout.ms", "3000") .option("kafka.default.api.timeout.ms", "3000") .option("startingOffsets", "earliest") .option("subscribePattern", s"$topicPrefix-.*") @@ -589,6 +591,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") + .option("kafka.request.timeout.ms", "3000") .option("kafka.default.api.timeout.ms", "3000") .option("subscribe", topic) // If a topic is deleted and we try to poll data starting from offset 0, @@ -1860,6 +1863,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest { .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", "stress.*") .option("failOnDataLoss", "false") + .option("kafka.request.timeout.ms", "3000") .option("kafka.default.api.timeout.ms", "3000") .load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 275a8170182fe..e4a1e4f77b28e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -462,32 +462,24 @@ class KafkaTestUtils( server.logManager.cleanupLogs() } + private def getOffsets(topics: Set[String], offsetSpec: OffsetSpec): Map[TopicPartition, Long] = { + val listOffsetsParams = adminClient.describeTopics(topics.asJava).all().get().asScala + .flatMap { topicDescription => + topicDescription._2.partitions().asScala.map { topicPartitionInfo => + new TopicPartition(topicDescription._1, topicPartitionInfo.partition()) + } + }.map(_ -> offsetSpec).toMap.asJava + val partitionOffsets = adminClient.listOffsets(listOffsetsParams).all().get().asScala + .map(result => result._1 -> result._2.offset()).toMap + partitionOffsets + } + def getEarliestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { - val kc = new KafkaConsumer[String, String](consumerConfiguration) - logInfo("Created consumer to get earliest offsets") - kc.subscribe(topics.asJavaCollection) - kc.poll(0) - val partitions = kc.assignment() - kc.pause(partitions) - kc.seekToBeginning(partitions) - val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap - kc.close() - logInfo("Closed consumer to get earliest offsets") - offsets + getOffsets(topics, OffsetSpec.earliest()) } def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { - val kc = new KafkaConsumer[String, String](consumerConfiguration) - logInfo("Created consumer to get latest offsets") - kc.subscribe(topics.asJavaCollection) - kc.poll(0) - val partitions = kc.assignment() - kc.pause(partitions) - kc.seekToEnd(partitions) - val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap - kc.close() - logInfo("Closed consumer to get latest offsets") - offsets + getOffsets(topics, OffsetSpec.latest()) } def listConsumerGroups(): ListConsumerGroupsResult = { @@ -559,17 +551,6 @@ class KafkaTestUtils( } } - private def consumerConfiguration: Properties = { - val props = new Properties() - props.put("bootstrap.servers", brokerAddress) - props.put("group.id", "group-KafkaTestUtils-" + Random.nextInt) - props.put("value.deserializer", classOf[StringDeserializer].getName) - props.put("key.deserializer", classOf[StringDeserializer].getName) - props.put("enable.auto.commit", "false") - setAuthenticationConfigIfNeeded(props) - props - } - private def setAuthenticationConfigIfNeeded(props: Properties): Unit = { if (secure) { val jaasParams = KafkaTokenUtil.getKeytabJaasParams( diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 2ac83c8ee6b38..4dacc03c72719 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -97,7 +97,7 @@ private[kinesis] class KinesisTestUtils(streamShardCount: Int = 2) extends Loggi } def getShards(): Seq[Shard] = { - kinesisClient.describeStream(_streamName).getStreamDescription.getShards.asScala + kinesisClient.describeStream(_streamName).getStreamDescription.getShards.asScala.toSeq } def splitShard(shardId: String): Unit = { @@ -137,7 +137,7 @@ private[kinesis] class KinesisTestUtils(streamShardCount: Int = 2) extends Loggi * Expose a Python friendly API. */ def pushData(testData: java.util.List[Int]): Unit = { - pushData(testData.asScala, aggregate = false) + pushData(testData.asScala.toSeq, aggregate = false) } def deleteStream(): Unit = { @@ -289,6 +289,6 @@ private[kinesis] class SimpleDataGenerator( sentSeqNumbers += ((num, seqNumber)) } - shardIdToSeqNumbers.toMap + shardIdToSeqNumbers.mapValues(_.toSeq).toMap } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala index af84498d5e47e..c76eb7c29dd94 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala @@ -72,6 +72,6 @@ private[kinesis] class KPLDataGenerator(regionName: String) extends KinesisDataG Futures.addCallback(future, kinesisCallBack, ThreadUtils.sameThreadExecutorService) } producer.flushSync() - shardIdToSeqNumbers.toMap + shardIdToSeqNumbers.mapValues(_.toSeq).toMap } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala index 2c7b9c58e6fa6..12d950096b4c2 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala @@ -47,8 +47,8 @@ abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) require(shardIdToDataAndSeqNumbers.size > 1, "Need data to be sent to multiple shards") shardIds = shardIdToDataAndSeqNumbers.keySet.toSeq - shardIdToData = shardIdToDataAndSeqNumbers.mapValues { _.map { _._1 }} - shardIdToSeqNumbers = shardIdToDataAndSeqNumbers.mapValues { _.map { _._2 }} + shardIdToData = shardIdToDataAndSeqNumbers.mapValues(_.map(_._1)).toMap + shardIdToSeqNumbers = shardIdToDataAndSeqNumbers.mapValues(_.map(_._2)).toMap shardIdToRange = shardIdToSeqNumbers.map { case (shardId, seqNumbers) => val seqNumRange = SequenceNumberRange( testUtils.streamName, shardId, seqNumbers.head, seqNumbers.last, seqNumbers.size) diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index 470a8cecc8fd9..d008de3b3f1c4 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -26,7 +26,9 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason import com.amazonaws.services.kinesis.model.Record import org.mockito.ArgumentMatchers.{anyList, anyString, eq => meq} import org.mockito.Mockito.{never, times, verify, when} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.streaming.{Duration, TestSuiteBase} diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index eee62d25e62bb..647f17859d041 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -23,8 +23,8 @@ import scala.util.Random import com.amazonaws.services.kinesis.model.Record import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} -import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.network.util.JavaUtils diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala index 994395bbffa56..f909b703ac6e0 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala @@ -34,7 +34,7 @@ class ShortestPathsSuite extends SparkFunSuite with LocalSparkContext { val graph = Graph.fromEdgeTuples(edges, 1) val landmarks = Seq(1, 4).map(_.toLong) val results = ShortestPaths.run(graph, landmarks).vertices.collect.map { - case (v, spMap) => (v, spMap.mapValues(i => i)) + case (v, spMap) => (v, spMap.mapValues(i => i).toMap) } assert(results.toSet === shortestPaths) } diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 3ae4633c79b04..778fd46b91fa1 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -235,20 +235,17 @@ String getScalaVersion() { return scala; } String sparkHome = getSparkHome(); - // TODO: revisit for Scala 2.13 support File scala212 = new File(sparkHome, "launcher/target/scala-2.12"); - // File scala211 = new File(sparkHome, "launcher/target/scala-2.11"); - // checkState(!scala212.isDirectory() || !scala211.isDirectory(), - // "Presence of build for multiple Scala versions detected.\n" + - // "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); - // if (scala212.isDirectory()) { - // return "2.12"; - // } else { - // checkState(scala211.isDirectory(), "Cannot find any build directories."); - // return "2.11"; - // } - checkState(scala212.isDirectory(), "Cannot find any build directories."); - return "2.12"; + File scala213 = new File(sparkHome, "launcher/target/scala-2.13"); + checkState(!scala212.isDirectory() || !scala213.isDirectory(), + "Presence of build for multiple Scala versions detected.\n" + + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + if (scala213.isDirectory()) { + return "2.13"; + } else { + checkState(scala212.isDirectory(), "Cannot find any build directories."); + return "2.12"; + } } String getSparkHome() { diff --git a/licenses-binary/LICENSE-heapq.txt b/licenses-binary/LICENSE-heapq.txt deleted file mode 100644 index 0c4c4b954bea4..0000000000000 --- a/licenses-binary/LICENSE-heapq.txt +++ /dev/null @@ -1,280 +0,0 @@ - -# A. HISTORY OF THE SOFTWARE -# ========================== -# -# Python was created in the early 1990s by Guido van Rossum at Stichting -# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands -# as a successor of a language called ABC. Guido remains Python's -# principal author, although it includes many contributions from others. -# -# In 1995, Guido continued his work on Python at the Corporation for -# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) -# in Reston, Virginia where he released several versions of the -# software. -# -# In May 2000, Guido and the Python core development team moved to -# BeOpen.com to form the BeOpen PythonLabs team. In October of the same -# year, the PythonLabs team moved to Digital Creations (now Zope -# Corporation, see http://www.zope.com). In 2001, the Python Software -# Foundation (PSF, see http://www.python.org/psf/) was formed, a -# non-profit organization created specifically to own Python-related -# Intellectual Property. Zope Corporation is a sponsoring member of -# the PSF. -# -# All Python releases are Open Source (see http://www.opensource.org for -# the Open Source Definition). Historically, most, but not all, Python -# releases have also been GPL-compatible; the table below summarizes -# the various releases. -# -# Release Derived Year Owner GPL- -# from compatible? (1) -# -# 0.9.0 thru 1.2 1991-1995 CWI yes -# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes -# 1.6 1.5.2 2000 CNRI no -# 2.0 1.6 2000 BeOpen.com no -# 1.6.1 1.6 2001 CNRI yes (2) -# 2.1 2.0+1.6.1 2001 PSF no -# 2.0.1 2.0+1.6.1 2001 PSF yes -# 2.1.1 2.1+2.0.1 2001 PSF yes -# 2.2 2.1.1 2001 PSF yes -# 2.1.2 2.1.1 2002 PSF yes -# 2.1.3 2.1.2 2002 PSF yes -# 2.2.1 2.2 2002 PSF yes -# 2.2.2 2.2.1 2002 PSF yes -# 2.2.3 2.2.2 2003 PSF yes -# 2.3 2.2.2 2002-2003 PSF yes -# 2.3.1 2.3 2002-2003 PSF yes -# 2.3.2 2.3.1 2002-2003 PSF yes -# 2.3.3 2.3.2 2002-2003 PSF yes -# 2.3.4 2.3.3 2004 PSF yes -# 2.3.5 2.3.4 2005 PSF yes -# 2.4 2.3 2004 PSF yes -# 2.4.1 2.4 2005 PSF yes -# 2.4.2 2.4.1 2005 PSF yes -# 2.4.3 2.4.2 2006 PSF yes -# 2.4.4 2.4.3 2006 PSF yes -# 2.5 2.4 2006 PSF yes -# 2.5.1 2.5 2007 PSF yes -# 2.5.2 2.5.1 2008 PSF yes -# 2.5.3 2.5.2 2008 PSF yes -# 2.6 2.5 2008 PSF yes -# 2.6.1 2.6 2008 PSF yes -# 2.6.2 2.6.1 2009 PSF yes -# 2.6.3 2.6.2 2009 PSF yes -# 2.6.4 2.6.3 2009 PSF yes -# 2.6.5 2.6.4 2010 PSF yes -# 2.7 2.6 2010 PSF yes -# -# Footnotes: -# -# (1) GPL-compatible doesn't mean that we're distributing Python under -# the GPL. All Python licenses, unlike the GPL, let you distribute -# a modified version without making your changes open source. The -# GPL-compatible licenses make it possible to combine Python with -# other software that is released under the GPL; the others don't. -# -# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, -# because its license has a choice of law clause. According to -# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 -# is "not incompatible" with the GPL. -# -# Thanks to the many outside volunteers who have worked under Guido's -# direction to make these releases possible. -# -# -# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON -# =============================================================== -# -# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 -# -------------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Python Software Foundation -# ("PSF"), and the Individual or Organization ("Licensee") accessing and -# otherwise using this software ("Python") in source or binary form and -# its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, PSF hereby -# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, -# analyze, test, perform and/or display publicly, prepare derivative works, -# distribute, and otherwise use Python alone or in any derivative version, -# provided, however, that PSF's License Agreement and PSF's notice of copyright, -# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, -# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained -# in Python alone or in any derivative version prepared by Licensee. -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python. -# -# 4. PSF is making Python available to Licensee on an "AS IS" -# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. Nothing in this License Agreement shall be deemed to create any -# relationship of agency, partnership, or joint venture between PSF and -# Licensee. This License Agreement does not grant permission to use PSF -# trademarks or trade name in a trademark sense to endorse or promote -# products or services of Licensee, or any third party. -# -# 8. By copying, installing or otherwise using Python, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 -# ------------------------------------------- -# -# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 -# -# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an -# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the -# Individual or Organization ("Licensee") accessing and otherwise using -# this software in source or binary form and its associated -# documentation ("the Software"). -# -# 2. Subject to the terms and conditions of this BeOpen Python License -# Agreement, BeOpen hereby grants Licensee a non-exclusive, -# royalty-free, world-wide license to reproduce, analyze, test, perform -# and/or display publicly, prepare derivative works, distribute, and -# otherwise use the Software alone or in any derivative version, -# provided, however, that the BeOpen Python License is retained in the -# Software, alone or in any derivative version prepared by Licensee. -# -# 3. BeOpen is making the Software available to Licensee on an "AS IS" -# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE -# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS -# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY -# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 5. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 6. This License Agreement shall be governed by and interpreted in all -# respects by the law of the State of California, excluding conflict of -# law provisions. Nothing in this License Agreement shall be deemed to -# create any relationship of agency, partnership, or joint venture -# between BeOpen and Licensee. This License Agreement does not grant -# permission to use BeOpen trademarks or trade names in a trademark -# sense to endorse or promote products or services of Licensee, or any -# third party. As an exception, the "BeOpen Python" logos available at -# http://www.pythonlabs.com/logos.html may be used according to the -# permissions granted on that web page. -# -# 7. By copying, installing or otherwise using the software, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 -# --------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Corporation for National -# Research Initiatives, having an office at 1895 Preston White Drive, -# Reston, VA 20191 ("CNRI"), and the Individual or Organization -# ("Licensee") accessing and otherwise using Python 1.6.1 software in -# source or binary form and its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, CNRI -# hereby grants Licensee a nonexclusive, royalty-free, world-wide -# license to reproduce, analyze, test, perform and/or display publicly, -# prepare derivative works, distribute, and otherwise use Python 1.6.1 -# alone or in any derivative version, provided, however, that CNRI's -# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) -# 1995-2001 Corporation for National Research Initiatives; All Rights -# Reserved" are retained in Python 1.6.1 alone or in any derivative -# version prepared by Licensee. Alternately, in lieu of CNRI's License -# Agreement, Licensee may substitute the following text (omitting the -# quotes): "Python 1.6.1 is made available subject to the terms and -# conditions in CNRI's License Agreement. This Agreement together with -# Python 1.6.1 may be located on the Internet using the following -# unique, persistent identifier (known as a handle): 1895.22/1013. This -# Agreement may also be obtained from a proxy server on the Internet -# using the following URL: http://hdl.handle.net/1895.22/1013". -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python 1.6.1 or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python 1.6.1. -# -# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" -# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. This License Agreement shall be governed by the federal -# intellectual property law of the United States, including without -# limitation the federal copyright law, and, to the extent such -# U.S. federal law does not apply, by the law of the Commonwealth of -# Virginia, excluding Virginia's conflict of law provisions. -# Notwithstanding the foregoing, with regard to derivative works based -# on Python 1.6.1 that incorporate non-separable material that was -# previously distributed under the GNU General Public License (GPL), the -# law of the Commonwealth of Virginia shall govern this License -# Agreement only as to issues arising under or with respect to -# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this -# License Agreement shall be deemed to create any relationship of -# agency, partnership, or joint venture between CNRI and Licensee. This -# License Agreement does not grant permission to use CNRI trademarks or -# trade name in a trademark sense to endorse or promote products or -# services of Licensee, or any third party. -# -# 8. By clicking on the "ACCEPT" button where indicated, or by copying, -# installing or otherwise using Python 1.6.1, Licensee agrees to be -# bound by the terms and conditions of this License Agreement. -# -# ACCEPT -# -# -# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 -# -------------------------------------------------- -# -# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, -# The Netherlands. All rights reserved. -# -# Permission to use, copy, modify, and distribute this software and its -# documentation for any purpose and without fee is hereby granted, -# provided that the above copyright notice appear in all copies and that -# both that copyright notice and this permission notice appear in -# supporting documentation, and that the name of Stichting Mathematisch -# Centrum or CWI not be used in advertising or publicity pertaining to -# distribution of the software without specific, written prior -# permission. -# -# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO -# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND -# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE -# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES -# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN -# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT -# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE-heapq.txt b/licenses/LICENSE-heapq.txt deleted file mode 100644 index 45be6b83a53be..0000000000000 --- a/licenses/LICENSE-heapq.txt +++ /dev/null @@ -1,49 +0,0 @@ -PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 --------------------------------------------- - -1. This LICENSE AGREEMENT is between the Python Software Foundation -("PSF"), and the Individual or Organization ("Licensee") accessing and -otherwise using this software ("Python") in source or binary form and -its associated documentation. - -2. Subject to the terms and conditions of this License Agreement, PSF hereby -grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, -analyze, test, perform and/or display publicly, prepare derivative works, -distribute, and otherwise use Python alone or in any derivative version, -provided, however, that PSF's License Agreement and PSF's notice of copyright, -i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, -2011, 2012, 2013, 2014, 2015, 2016, 2017, 2018, 2019 Python Software Foundation; -All Rights Reserved" are retained in Python alone or in any derivative version -prepared by Licensee. - -3. In the event Licensee prepares a derivative work that is based on -or incorporates Python or any part thereof, and wants to make -the derivative work available to others as provided herein, then -Licensee hereby agrees to include in any such work a brief summary of -the changes made to Python. - -4. PSF is making Python available to Licensee on an "AS IS" -basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND -DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT -INFRINGE ANY THIRD PARTY RIGHTS. - -5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, -OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. - -6. This License Agreement will automatically terminate upon a material -breach of its terms and conditions. - -7. Nothing in this License Agreement shall be deemed to create any -relationship of agency, partnership, or joint venture between PSF and -Licensee. This License Agreement does not grant permission to use PSF -trademarks or trade name in a trademark sense to endorse or promote -products or services of Licensee, or any third party. - -8. By copying, installing or otherwise using Python, Licensee -agrees to be bound by the terms and conditions of this License -Agreement. - diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala index cb3b56bba87b0..d5ba1f906d389 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml // scalastyle:off -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite /** * Base abstract class for all unit tests in Spark for handling common functionality. */ private[spark] abstract class SparkMLFunSuite - extends FunSuite + extends AnyFunSuite with BeforeAndAfterAll { // scalastyle:on } diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 2b4b0fc55b955..253d4083de7d4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -838,7 +838,7 @@ private[ml] class FeedForwardTrainer( * @param data RDD of input and output vector pairs * @return model */ - def train(data: RDD[(Vector, Vector)]): TopologyModel = { + def train(data: RDD[(Vector, Vector)]): (TopologyModel, Array[Double]) = { val w = if (getWeights == null) { // TODO: will make a copy if vector is a subvector of BDV (see Vectors code) topology.model(_seed).weights @@ -851,9 +851,14 @@ private[ml] class FeedForwardTrainer( } val handlePersistence = trainData.getStorageLevel == StorageLevel.NONE if (handlePersistence) trainData.persist(StorageLevel.MEMORY_AND_DISK) - val newWeights = optimizer.optimize(trainData, w) + val (newWeights, lossHistory) = optimizer match { + case lbfgs: LBFGS => lbfgs.optimizeWithLossReturned(trainData, w) + case sgd: GradientDescent => sgd.optimizeWithLossReturned(trainData, w) + case other => throw new UnsupportedOperationException( + s"Only LBFGS and GradientDescent are supported but got ${other.getClass}.") + } if (handlePersistence) trainData.unpersist() - topology.model(newWeights) + (topology.model(newWeights), lossHistory) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 530943c910d7d..b2c63cbcb464b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -129,7 +129,7 @@ class DecisionTreeClassifier @Since("1.4.0") ( instr.logNumClasses(numClasses) instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, probabilityCol, leafCol, maxDepth, maxBins, minInstancesPerNode, minInfoGain, - maxMemoryInMB, cacheNodeIds, checkpointInterval, impurity, seed) + maxMemoryInMB, cacheNodeIds, checkpointInterval, impurity, seed, thresholds) val trees = RandomForest.run(instances, strategy, numTrees = 1, featureSubsetStrategy = "all", seed = $(seed), instr = Some(instr), parentUID = Some(uid)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala index cc691d1c0c58c..4188f6893ea37 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala @@ -85,7 +85,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setFactorSize(value: Int): this.type = set(factorSize, value) - setDefault(factorSize -> 8) /** * Set whether to fit intercept term. @@ -95,7 +94,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Set whether to fit linear term. @@ -105,7 +103,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setFitLinear(value: Boolean): this.type = set(fitLinear, value) - setDefault(fitLinear -> true) /** * Set the L2 regularization parameter. @@ -115,7 +112,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set the mini-batch fraction parameter. @@ -125,7 +121,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setMiniBatchFraction(value: Double): this.type = set(miniBatchFraction, value) - setDefault(miniBatchFraction -> 1.0) /** * Set the standard deviation of initial coefficients. @@ -135,7 +130,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setInitStd(value: Double): this.type = set(initStd, value) - setDefault(initStd -> 0.01) /** * Set the maximum number of iterations. @@ -145,7 +139,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the initial step size for the first step (like learning rate). @@ -155,7 +148,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(stepSize -> 1.0) /** * Set the convergence tolerance of iterations. @@ -165,7 +157,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Set the solver algorithm used for optimization. @@ -176,7 +167,6 @@ class FMClassifier @Since("3.0.0") ( */ @Since("3.0.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> AdamW) /** * Set the random seed for weight initialization. @@ -187,9 +177,7 @@ class FMClassifier @Since("3.0.0") ( def setSeed(value: Long): this.type = set(seed, value) override protected def train( - dataset: Dataset[_] - ): FMClassificationModel = instrumented { instr => - + dataset: Dataset[_]): FMClassificationModel = instrumented { instr => val numClasses = 2 if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + @@ -200,7 +188,7 @@ class FMClassifier @Since("3.0.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, factorSize, fitIntercept, fitLinear, regParam, - miniBatchFraction, initStd, maxIter, stepSize, tol, solver) + miniBatchFraction, initStd, maxIter, stepSize, tol, solver, thresholds) instr.logNumClasses(numClasses) val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 46810bccc8e69..ca43ef863003a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -197,7 +197,7 @@ class GBTClassifier @Since("1.4.0") ( instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, leafCol, impurity, lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, minWeightFractionPerNode, seed, stepSize, subsamplingRate, cacheNodeIds, - checkpointInterval, featureSubsetStrategy, validationIndicatorCol, validationTol) + checkpointInterval, featureSubsetStrategy, validationIndicatorCol, validationTol, thresholds) instr.logNumClasses(numClasses) val categoricalFeatures = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 4adc527c89b36..77272c65eb231 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -55,6 +55,9 @@ private[classification] trait LinearSVCParams extends ClassifierParams with HasR */ final override val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction applied to rawPrediction") + + setDefault(regParam -> 0.0, maxIter -> 100, fitIntercept -> true, tol -> 1E-6, + standardization -> true, threshold -> 0.0, aggregationDepth -> 2, blockSize -> 1) } /** @@ -82,7 +85,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set the maximum number of iterations. @@ -92,7 +94,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Whether to fit an intercept term. @@ -102,7 +103,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Set the convergence tolerance of iterations. @@ -113,7 +113,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Whether to standardize the training features before fitting the model. @@ -123,7 +122,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setStandardization(value: Boolean): this.type = set(standardization, value) - setDefault(standardization -> true) /** * Set the value of param [[weightCol]]. @@ -142,7 +140,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setThreshold(value: Double): this.type = set(threshold, value) - setDefault(threshold -> 0.0) /** * Suggested depth for treeAggregate (greater than or equal to 2). @@ -154,7 +151,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("2.2.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Set block size for stacking input data in matrices. @@ -173,7 +169,6 @@ class LinearSVC @Since("2.2.0") ( */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) @Since("2.2.0") override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) @@ -381,7 +376,6 @@ class LinearSVCModel private[classification] ( @Since("2.2.0") def setThreshold(value: Double): this.type = set(threshold, value) - setDefault(threshold, 0.0) private val margin: Vector => Double = (features) => { BLAS.dot(features, coefficients) + intercept diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 47b3e2de7695c..4d763cbd29d3c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -243,6 +243,10 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas @Since("2.2.0") def getUpperBoundsOnIntercepts: Vector = $(upperBoundsOnIntercepts) + setDefault(regParam -> 0.0, elasticNetParam -> 0.0, maxIter -> 100, tol -> 1E-6, + fitIntercept -> true, family -> "auto", standardization -> true, threshold -> 0.5, + aggregationDepth -> 2, blockSize -> 1) + protected def usingBoundConstrainedOptimization: Boolean = { isSet(lowerBoundsOnCoefficients) || isSet(upperBoundsOnCoefficients) || isSet(lowerBoundsOnIntercepts) || isSet(upperBoundsOnIntercepts) @@ -290,7 +294,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.2.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set the ElasticNet mixing parameter. @@ -306,7 +309,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.4.0") def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - setDefault(elasticNetParam -> 0.0) /** * Set the maximum number of iterations. @@ -316,7 +318,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.2.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the convergence tolerance of iterations. @@ -327,7 +328,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.4.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Whether to fit an intercept term. @@ -337,7 +337,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.4.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Sets the value of param [[family]]. @@ -347,7 +346,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("2.1.0") def setFamily(value: String): this.type = set(family, value) - setDefault(family -> "auto") /** * Whether to standardize the training features before fitting the model. @@ -361,11 +359,9 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("1.5.0") def setStandardization(value: Boolean): this.type = set(standardization, value) - setDefault(standardization -> true) @Since("1.5.0") override def setThreshold(value: Double): this.type = super.setThreshold(value) - setDefault(threshold -> 0.5) @Since("1.5.0") override def getThreshold: Double = super.getThreshold @@ -396,7 +392,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("2.1.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Set the lower bounds on coefficients if fitting under bound constrained optimization. @@ -447,7 +442,6 @@ class LogisticRegression @Since("1.2.0") ( */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) private def assertBoundConstrainedOptimizationParamsValid( numCoefficientSets: Int, @@ -512,8 +506,8 @@ class LogisticRegression @Since("1.2.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, - probabilityCol, regParam, elasticNetParam, standardization, threshold, maxIter, tol, - fitIntercept, blockSize) + probabilityCol, regParam, elasticNetParam, standardization, threshold, thresholds, maxIter, + tol, fitIntercept, blockSize) val instances = extractInstances(dataset) .setName("training instances") @@ -1104,16 +1098,42 @@ class LogisticRegressionModel private[spark] ( _intercept } - private lazy val _intercept = interceptVector.toArray.head + private lazy val _intercept = interceptVector(0) + private lazy val _interceptVector = interceptVector.toDense + private var _threshold = Double.NaN + private var _rawThreshold = Double.NaN + + updateBinaryThreshold() + + private def updateBinaryThreshold(): Unit = { + if (!isMultinomial) { + _threshold = getThreshold + if (_threshold == 0.0) { + _rawThreshold = Double.NegativeInfinity + } else if (_threshold == 1.0) { + _rawThreshold = Double.PositiveInfinity + } else { + _rawThreshold = math.log(_threshold / (1.0 - _threshold)) + } + } + } @Since("1.5.0") - override def setThreshold(value: Double): this.type = super.setThreshold(value) + override def setThreshold(value: Double): this.type = { + super.setThreshold(value) + updateBinaryThreshold() + this + } @Since("1.5.0") override def getThreshold: Double = super.getThreshold @Since("1.5.0") - override def setThresholds(value: Array[Double]): this.type = super.setThresholds(value) + override def setThresholds(value: Array[Double]): this.type = { + super.setThresholds(value) + updateBinaryThreshold() + this + } @Since("1.5.0") override def getThresholds: Array[Double] = super.getThresholds @@ -1125,7 +1145,7 @@ class LogisticRegressionModel private[spark] ( /** Margin (rawPrediction) for each class label. */ private val margins: Vector => Vector = (features) => { - val m = interceptVector.toDense.copy + val m = _interceptVector.copy BLAS.gemv(1.0, coefficientMatrix, features, 1.0, m) m } @@ -1184,52 +1204,43 @@ class LogisticRegressionModel private[spark] ( override def predict(features: Vector): Double = if (isMultinomial) { super.predict(features) } else { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - if (score(features) > getThreshold) 1 else 0 + // Note: We should use _threshold instead of $(threshold) since getThreshold is overridden. + if (score(features) > _threshold) 1 else 0 } override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { rawPrediction match { case dv: DenseVector => + val values = dv.values if (isMultinomial) { - val size = dv.size - val values = dv.values - // get the maximum margin val maxMarginIndex = rawPrediction.argmax val maxMargin = rawPrediction(maxMarginIndex) if (maxMargin == Double.PositiveInfinity) { var k = 0 - while (k < size) { + while (k < numClasses) { values(k) = if (k == maxMarginIndex) 1.0 else 0.0 k += 1 } } else { - val sum = { - var temp = 0.0 - var k = 0 - while (k < numClasses) { - values(k) = if (maxMargin > 0) { - math.exp(values(k) - maxMargin) - } else { - math.exp(values(k)) - } - temp += values(k) - k += 1 + var sum = 0.0 + var k = 0 + while (k < numClasses) { + values(k) = if (maxMargin > 0) { + math.exp(values(k) - maxMargin) + } else { + math.exp(values(k)) } - temp + sum += values(k) + k += 1 } BLAS.scal(1 / sum, dv) } dv } else { - var i = 0 - val size = dv.size - while (i < size) { - dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i))) - i += 1 - } + values(0) = 1.0 / (1.0 + math.exp(-values(0))) + values(1) = 1.0 - values(0) dv } case sv: SparseVector => @@ -1259,16 +1270,8 @@ class LogisticRegressionModel private[spark] ( if (isMultinomial) { super.raw2prediction(rawPrediction) } else { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - val t = getThreshold - val rawThreshold = if (t == 0.0) { - Double.NegativeInfinity - } else if (t == 1.0) { - Double.PositiveInfinity - } else { - math.log(t / (1.0 - t)) - } - if (rawPrediction(1) > rawThreshold) 1 else 0 + // Note: We should use _threshold instead of $(threshold) since getThreshold is overridden. + if (rawPrediction(1) > _rawThreshold) 1.0 else 0.0 } } @@ -1276,8 +1279,8 @@ class LogisticRegressionModel private[spark] ( if (isMultinomial) { super.probability2prediction(probability) } else { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - if (probability(1) > getThreshold) 1 else 0 + // Note: We should use _threshold instead of $(threshold) since getThreshold is overridden. + if (probability(1) > _threshold) 1.0 else 0.0 } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 6bffc372b68fe..58fc53517c9b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -27,7 +27,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql._ import org.apache.spark.util.VersionUtils.majorMinorVersion /** Params for Multilayer Perceptron. */ @@ -185,7 +185,7 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, layers, maxIter, - tol, blockSize, solver, stepSize, seed) + tol, blockSize, solver, stepSize, seed, thresholds) val myLayers = $(layers) val labels = myLayers.last @@ -225,8 +225,24 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( s"The solver $solver is not supported by MultilayerPerceptronClassifier.") } trainer.setStackSize($(blockSize)) - val mlpModel = trainer.train(data) - new MultilayerPerceptronClassificationModel(uid, mlpModel.weights) + val (mlpModel, objectiveHistory) = trainer.train(data) + createModel(dataset, mlpModel.weights, objectiveHistory) + } + + private def createModel( + dataset: Dataset[_], + weights: Vector, + objectiveHistory: Array[Double]): MultilayerPerceptronClassificationModel = { + val model = copyValues(new MultilayerPerceptronClassificationModel(uid, weights)) + + val (summaryModel, _, predictionColName) = model.findSummaryModel() + val summary = new MultilayerPerceptronClassificationTrainingSummaryImpl( + summaryModel.transform(dataset), + predictionColName, + $(labelCol), + "", + objectiveHistory) + model.setSummary(Some(summary)) } } @@ -259,7 +275,8 @@ class MultilayerPerceptronClassificationModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("2.0.0") val weights: Vector) extends ProbabilisticClassificationModel[Vector, MultilayerPerceptronClassificationModel] - with MultilayerPerceptronParams with Serializable with MLWritable { + with MultilayerPerceptronParams with Serializable with MLWritable + with HasTrainingSummary[MultilayerPerceptronClassificationTrainingSummary]{ @Since("1.6.0") override lazy val numFeatures: Int = $(layers).head @@ -268,6 +285,26 @@ class MultilayerPerceptronClassificationModel private[ml] ( .multiLayerPerceptron($(layers), softmaxOnTop = true) .model(weights) + /** + * Gets summary of model on training set. An exception is thrown + * if `hasSummary` is false. + */ + @Since("3.1.0") + override def summary: MultilayerPerceptronClassificationTrainingSummary = super.summary + + /** + * Evaluates the model on a test dataset. + * + * @param dataset Test dataset to evaluate model on. + */ + @Since("3.1.0") + def evaluate(dataset: Dataset[_]): MultilayerPerceptronClassificationSummary = { + // Handle possible missing or invalid probability or prediction columns + val (summaryModel, _, predictionColName) = findSummaryModel() + new MultilayerPerceptronClassificationSummaryImpl(summaryModel.transform(dataset), + predictionColName, $(labelCol), "") + } + /** * Predict label for the given features. * This internal method is used to implement `transform()` and output [[predictionCol]]. @@ -359,3 +396,51 @@ object MultilayerPerceptronClassificationModel } } } + + +/** + * Abstraction for MultilayerPerceptronClassification results for a given model. + */ +sealed trait MultilayerPerceptronClassificationSummary extends ClassificationSummary + +/** + * Abstraction for MultilayerPerceptronClassification training results. + */ +sealed trait MultilayerPerceptronClassificationTrainingSummary + extends MultilayerPerceptronClassificationSummary with TrainingSummary + +/** + * MultilayerPerceptronClassification training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + */ +private class MultilayerPerceptronClassificationTrainingSummaryImpl( + predictions: DataFrame, + predictionCol: String, + labelCol: String, + weightCol: String, + override val objectiveHistory: Array[Double]) + extends MultilayerPerceptronClassificationSummaryImpl( + predictions, predictionCol, labelCol, weightCol) + with MultilayerPerceptronClassificationTrainingSummary + +/** + * MultilayerPerceptronClassification results for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param weightCol field in "predictions" which gives the weight of each instance. + */ +private class MultilayerPerceptronClassificationSummaryImpl( + @transient override val predictions: DataFrame, + override val predictionCol: String, + override val labelCol: String, + override val weightCol: String) + extends MultilayerPerceptronClassificationSummary diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index e65295dbdaf55..6b1537bcc5069 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -64,6 +64,8 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** @group getParam */ final def getModelType: String = $(modelType) + + setDefault(smoothing -> 1.0, modelType -> NaiveBayes.Multinomial) } // scalastyle:off line.size.limit @@ -107,7 +109,6 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setSmoothing(value: Double): this.type = set(smoothing, value) - setDefault(smoothing -> 1.0) /** * Set the model type using a string (case-sensitive). @@ -117,7 +118,6 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setModelType(value: String): this.type = set(modelType, value) - setDefault(modelType -> Multinomial) /** * Sets the value of param [[weightCol]]. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index b09f11dcfe156..5a60bed2652f7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -72,6 +72,8 @@ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter @Since("2.0.0") def getMinDivisibleClusterSize: Double = $(minDivisibleClusterSize) + setDefault(k -> 4, maxIter -> 20, minDivisibleClusterSize -> 1.0) + /** * Validates and transforms the input schema. * @param schema input schema @@ -226,11 +228,6 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.0.0") override val uid: String) extends Estimator[BisectingKMeansModel] with BisectingKMeansParams with DefaultParamsWritable { - setDefault( - k -> 4, - maxIter -> 20, - minDivisibleClusterSize -> 1.0) - @Since("2.0.0") override def copy(extra: ParamMap): BisectingKMeans = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 90845021fc073..7df87d2c3336a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -59,6 +59,8 @@ private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter w @Since("2.0.0") def getK: Int = $(k) + setDefault(k -> 2, maxIter -> 100, tol -> 0.01, blockSize -> 1) + /** * Validates and transforms the input schema. * @@ -328,11 +330,6 @@ class GaussianMixture @Since("2.0.0") ( @Since("2.0.0") override val uid: String) extends Estimator[GaussianMixtureModel] with GaussianMixtureParams with DefaultParamsWritable { - setDefault( - k -> 2, - maxIter -> 100, - tol -> 0.01) - @Since("2.0.0") override def copy(extra: ParamMap): GaussianMixture = defaultCopy(extra) @@ -392,7 +389,6 @@ class GaussianMixture @Since("2.0.0") ( */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) /** * Number of samples per cluster to use when initializing Gaussians. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 806015b633c23..5c06973e618bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -87,6 +87,9 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe @Since("1.5.0") def getInitSteps: Int = $(initSteps) + setDefault(k -> 2, maxIter -> 20, initMode -> MLlibKMeans.K_MEANS_PARALLEL, initSteps -> 2, + tol -> 1e-4, distanceMeasure -> DistanceMeasure.EUCLIDEAN) + /** * Validates and transforms the input schema. * @param schema input schema @@ -271,14 +274,6 @@ class KMeans @Since("1.5.0") ( @Since("1.5.0") override val uid: String) extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { - setDefault( - k -> 2, - maxIter -> 20, - initMode -> MLlibKMeans.K_MEANS_PARALLEL, - initSteps -> 2, - tol -> 1e-4, - distanceMeasure -> DistanceMeasure.EUCLIDEAN) - @Since("1.5.0") override def copy(extra: ParamMap): KMeans = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 4e86b3b247ace..c1b76fb40b2f6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -199,8 +199,6 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM " with estimates of the topic mixture distribution for each document (often called \"theta\"" + " in the literature). Returns a vector of zeros for an empty document.") - setDefault(topicDistributionCol -> "topicDistribution") - /** @group getParam */ @Since("1.6.0") def getTopicDistributionCol: String = $(topicDistributionCol) @@ -315,6 +313,11 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM @Since("2.0.0") def getKeepLastCheckpoint: Boolean = $(keepLastCheckpoint) + setDefault(maxIter -> 20, k -> 10, optimizer -> "online", checkpointInterval -> 10, + learningOffset -> 1024, learningDecay -> 0.51, subsamplingRate -> 0.05, + optimizeDocConcentration -> true, keepLastCheckpoint -> true, + topicDistributionCol -> "topicDistribution") + /** * Validates and transforms the input schema. * @@ -863,10 +866,6 @@ class LDA @Since("1.6.0") ( @Since("1.6.0") def this() = this(Identifiable.randomUID("lda")) - setDefault(maxIter -> 20, k -> 10, optimizer -> "online", checkpointInterval -> 10, - learningOffset -> 1024, learningDecay -> 0.51, subsamplingRate -> 0.05, - optimizeDocConcentration -> true, keepLastCheckpoint -> true) - /** * The features for LDA should be a `Vector` representing the word counts in a document. * The vector should be of length vocabSize, with counts for each term (word). diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala index 812a426a062c1..1466b32bef530 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala @@ -91,7 +91,7 @@ private[clustering] trait PowerIterationClusteringParams extends Params with Has @Since("2.4.0") def getDstCol: String = $(dstCol) - setDefault(srcCol -> "src", dstCol -> "dst") + setDefault(srcCol -> "src", dstCol -> "dst", k -> 2, maxIter -> 20, initMode -> "random") } /** @@ -111,11 +111,6 @@ class PowerIterationClustering private[clustering] ( @Since("2.4.0") override val uid: String) extends PowerIterationClusteringParams with DefaultParamsWritable { - setDefault( - k -> 2, - maxIter -> 20, - initMode -> "random") - @Since("2.4.0") def this() = this(Identifiable.randomUID("PowerIterationClustering")) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 52be22f714981..93b66f3ab7007 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -81,8 +81,6 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("3.0.0") def setNumBins(value: Int): this.type = set(numBins, value) - setDefault(numBins -> 1000) - /** @group setParam */ @Since("1.5.0") def setRawPredictionCol(value: String): this.type = set(rawPredictionCol, value) @@ -95,7 +93,7 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("3.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) - setDefault(metricName -> "areaUnderROC") + setDefault(metricName -> "areaUnderROC", numBins -> 1000) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 3d77792c4fc88..beeefde8c5fac 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -64,8 +64,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("1.5.0") def setMetricName(value: String): this.type = set(metricName, value) - setDefault(metricName -> "f1") - /** @group setParam */ @Since("1.5.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -105,8 +103,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setMetricLabel(value: Double): this.type = set(metricLabel, value) - setDefault(metricLabel -> 0.0) - /** * The beta value, which controls precision vs recall weighting, * used in `"weightedFMeasure"`, `"fMeasureByLabel"`. @@ -128,8 +124,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setBeta(value: Double): this.type = set(beta, value) - setDefault(beta -> 1.0) - /** * param for eps. log-loss is undefined for p=0 or p=1, so probabilities are clipped to * max(eps, min(1 - eps, p)). Must be in range (0, 0.5). The default value is 1e-15. @@ -150,7 +144,7 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setEps(value: Double): this.type = set(eps, value) - setDefault(eps -> 1e-15) + setDefault(metricName -> "f1", eps -> 1e-15, metricLabel -> 0.0, beta -> 1.0) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala index 1a82ac7a9472f..8ed26502407a8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala @@ -65,8 +65,6 @@ class MultilabelClassificationEvaluator @Since("3.0.0") (@Since("3.0.0") overrid @Since("3.0.0") def setMetricName(value: String): this.type = set(metricName, value) - setDefault(metricName -> "f1Measure") - /** * param for the class whose metric will be computed in `"precisionByLabel"`, `"recallByLabel"`, * `"f1MeasureByLabel"`. @@ -86,8 +84,6 @@ class MultilabelClassificationEvaluator @Since("3.0.0") (@Since("3.0.0") overrid /** @group setParam */ def setMetricLabel(value: Double): this.type = set(metricLabel, value) - setDefault(metricLabel -> 0.0) - /** @group setParam */ @Since("3.0.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -96,6 +92,8 @@ class MultilabelClassificationEvaluator @Since("3.0.0") (@Since("3.0.0") overrid @Since("3.0.0") def setLabelCol(value: String): this.type = set(labelCol, value) + setDefault(metricLabel -> 0.0, metricName -> "f1Measure") + @Since("3.0.0") override def evaluate(dataset: Dataset[_]): Double = { val metrics = getMetrics(dataset) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala index 82dda4109771d..01fb0599160b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala @@ -61,8 +61,6 @@ class RankingEvaluator @Since("3.0.0") (@Since("3.0.0") override val uid: String @Since("3.0.0") def setMetricName(value: String): this.type = set(metricName, value) - setDefault(metricName -> "meanAveragePrecision") - /** * param for ranking position value used in `"meanAveragePrecisionAtK"`, `"precisionAtK"`, * `"ndcgAtK"`, `"recallAtK"`. Must be > 0. The default value is 10. @@ -83,8 +81,6 @@ class RankingEvaluator @Since("3.0.0") (@Since("3.0.0") override val uid: String @Since("3.0.0") def setK(value: Int): this.type = set(k, value) - setDefault(k -> 10) - /** @group setParam */ @Since("3.0.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -93,6 +89,8 @@ class RankingEvaluator @Since("3.0.0") (@Since("3.0.0") override val uid: String @Since("3.0.0") def setLabelCol(value: String): this.type = set(labelCol, value) + setDefault(k -> 10, metricName -> "meanAveragePrecision") + @Since("3.0.0") override def evaluate(dataset: Dataset[_]): Double = { val metrics = getMetrics(dataset) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index f0b7c345c3285..902869cc681b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -80,8 +80,6 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("3.0.0") def setThroughOrigin(value: Boolean): this.type = set(throughOrigin, value) - setDefault(throughOrigin -> false) - /** @group setParam */ @Since("1.4.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -94,7 +92,7 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("3.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) - setDefault(metricName -> "rmse") + setDefault(metricName -> "rmse", throughOrigin -> false) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index 64f1722f5fcb8..ad1010da5c104 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -64,6 +64,8 @@ private[feature] trait ImputerParams extends Params with HasInputCol with HasInp /** @group getParam */ def getMissingValue: Double = $(missingValue) + setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) + /** Returns the input and output column names corresponding in pair. */ private[feature] def getInOutCols(): (Array[String], Array[String]) = { if (isSet(inputCol)) { @@ -144,8 +146,6 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) @Since("3.0.0") def setRelativeError(value: Double): this.type = set(relativeError, value) - setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) - override def fit(dataset: Dataset[_]): ImputerModel = { transformSchema(dataset.schema, logging = true) val spark = dataset.sparkSession diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 90187c331e835..22c4ca9cddf4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -58,6 +58,8 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H /** @group getParam */ def getMax: Double = $(max) + setDefault(min -> 0.0, max -> 1.0) + /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { require($(min) < $(max), s"The specified min(${$(min)}) is larger or equal to max(${$(max)})") @@ -93,8 +95,6 @@ class MinMaxScaler @Since("1.5.0") (@Since("1.5.0") override val uid: String) @Since("1.5.0") def this() = this(Identifiable.randomUID("minMaxScal")) - setDefault(min -> 0.0, max -> 1.0) - /** @group setParam */ @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 0ef092f6be463..5a500fefb57ec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -53,8 +53,6 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid "during fitting, invalid data will result in an error.", ParamValidators.inArray(OneHotEncoder.supportedHandleInvalids)) - setDefault(handleInvalid, OneHotEncoder.ERROR_INVALID) - /** * Whether to drop the last category in the encoded vector (default: true) * @group param @@ -62,12 +60,13 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid @Since("2.3.0") final val dropLast: BooleanParam = new BooleanParam(this, "dropLast", "whether to drop the last category") - setDefault(dropLast -> true) /** @group getParam */ @Since("2.3.0") def getDropLast: Boolean = $(dropLast) + setDefault(handleInvalid -> OneHotEncoder.ERROR_INVALID, dropLast -> true) + /** Returns the input and output column names corresponding in pair. */ private[feature] def getInOutCols(): (Array[String], Array[String]) = { if (isSet(inputCol)) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 4eedfc4dc0efa..b93c9b1fcd204 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -46,7 +46,6 @@ private[feature] trait QuantileDiscretizerBase extends Params val numBuckets = new IntParam(this, "numBuckets", "Number of buckets (quantiles, or " + "categories) into which data points are grouped. Must be >= 2.", ParamValidators.gtEq(2)) - setDefault(numBuckets -> 2) /** @group getParam */ def getNumBuckets: Int = getOrDefault(numBuckets) @@ -82,7 +81,8 @@ private[feature] trait QuantileDiscretizerBase extends Params "how to handle invalid entries. Options are skip (filter out rows with invalid values), " + "error (throw an error), or keep (keep invalid values in a special additional bucket).", ParamValidators.inArray(Bucketizer.supportedHandleInvalids)) - setDefault(handleInvalid, Bucketizer.ERROR_INVALID) + + setDefault(handleInvalid -> Bucketizer.ERROR_INVALID, numBuckets -> 2) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 7ccfafa4ac813..b8da020017f12 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -60,7 +60,6 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with @Since("2.1.0") val forceIndexLabel: BooleanParam = new BooleanParam(this, "forceIndexLabel", "Force to index label whether it is numeric or string") - setDefault(forceIndexLabel -> false) /** @group getParam */ @Since("2.1.0") @@ -80,7 +79,6 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with "type. Options are 'skip' (filter out rows with invalid data), error (throw an error), " + "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, StringIndexer.ERROR_INVALID) /** * Param for how to order categories of a string FEATURE column used by `StringIndexer`. @@ -113,12 +111,14 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with "The default value is 'frequencyDesc'. When the ordering is set to 'alphabetDesc', " + "RFormula drops the same category as R when encoding strings.", ParamValidators.inArray(StringIndexer.supportedStringOrderType)) - setDefault(stringIndexerOrderType, StringIndexer.frequencyDesc) /** @group getParam */ @Since("2.3.0") def getStringIndexerOrderType: String = $(stringIndexerOrderType) + setDefault(forceIndexLabel -> false, handleInvalid -> StringIndexer.ERROR_INVALID, + stringIndexerOrderType -> StringIndexer.frequencyDesc) + protected def hasLabelCol(schema: StructType): Boolean = { schema.map(_.name).contains($(labelCol)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index 72ab3dbc31016..e8f325ec58432 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -50,8 +50,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getLower: Double = $(lower) - setDefault(lower -> 0.25) - /** * Upper quantile to calculate quantile range, shared by all features * Default: 0.75 @@ -64,8 +62,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getUpper: Double = $(upper) - setDefault(upper -> 0.75) - /** * Whether to center the data with median before scaling. * It will build a dense output, so take care when applying to sparse input. @@ -78,8 +74,6 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getWithCentering: Boolean = $(withCentering) - setDefault(withCentering -> false) - /** * Whether to scale the data to quantile range. * Default: true @@ -91,7 +85,7 @@ private[feature] trait RobustScalerParams extends Params with HasInputCol with H /** @group getParam */ def getWithScaling: Boolean = $(withScaling) - setDefault(withScaling -> true) + setDefault(withScaling -> true, lower -> 0.25, upper -> 0.75, withCentering -> false) /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala index 627133968d149..46052a89fdf1a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Selector.scala @@ -50,7 +50,6 @@ private[feature] trait SelectorParams extends Params "Number of features that selector will select, ordered by ascending p-value. If the" + " number of features is < numTopFeatures, then this will select all features.", ParamValidators.gtEq(1)) - setDefault(numTopFeatures -> 50) /** @group getParam */ @Since("3.1.0") @@ -66,7 +65,6 @@ private[feature] trait SelectorParams extends Params final val percentile = new DoubleParam(this, "percentile", "Percentile of features that selector will select, ordered by ascending p-value.", ParamValidators.inRange(0, 1)) - setDefault(percentile -> 0.1) /** @group getParam */ @Since("3.1.0") @@ -81,7 +79,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fpr = new DoubleParam(this, "fpr", "The higest p-value for features to be kept.", ParamValidators.inRange(0, 1)) - setDefault(fpr -> 0.05) /** @group getParam */ @Since("3.1.0") @@ -96,7 +93,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fdr = new DoubleParam(this, "fdr", "The upper bound of the expected false discovery rate.", ParamValidators.inRange(0, 1)) - setDefault(fdr -> 0.05) /** @group getParam */ def getFdr: Double = $(fdr) @@ -110,7 +106,6 @@ private[feature] trait SelectorParams extends Params @Since("3.1.0") final val fwe = new DoubleParam(this, "fwe", "The upper bound of the expected family-wise error rate.", ParamValidators.inRange(0, 1)) - setDefault(fwe -> 0.05) /** @group getParam */ def getFwe: Double = $(fwe) @@ -125,12 +120,13 @@ private[feature] trait SelectorParams extends Params "The selector type. Supported options: numTopFeatures, percentile, fpr, fdr, fwe", ParamValidators.inArray(Array("numTopFeatures", "percentile", "fpr", "fdr", "fwe"))) - setDefault(selectorType -> "numTopFeatures") /** @group getParam */ @Since("3.1.0") def getSelectorType: String = $(selectorType) + setDefault(numTopFeatures -> 50, percentile -> 0.1, fpr -> 0.05, fdr -> 0.05, fwe -> 0.05, + selectorType -> "numTopFeatures") } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index be32f44287b6a..ab51fe6e78bd7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -56,8 +56,6 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, StringIndexer.ERROR_INVALID) - /** * Param for how to order labels of string column. The first label after ordering is assigned * an index of 0. @@ -80,6 +78,9 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi s"Supported options: ${StringIndexer.supportedStringOrderType.mkString(", ")}.", ParamValidators.inArray(StringIndexer.supportedStringOrderType)) + setDefault(handleInvalid -> StringIndexer.ERROR_INVALID, + stringOrderType -> StringIndexer.frequencyDesc) + /** @group getParam */ @Since("2.3.0") def getStringOrderType: String = $(stringOrderType) @@ -155,7 +156,6 @@ class StringIndexer @Since("1.4.0") ( /** @group setParam */ @Since("2.3.0") def setStringOrderType(value: String): this.type = set(stringOrderType, value) - setDefault(stringOrderType, StringIndexer.frequencyDesc) /** @group setParam */ @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 866074fb1453e..b7cf4392cd177 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -60,8 +60,6 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu "number of categories of the feature).", ParamValidators.inArray(VectorIndexer.supportedHandleInvalids)) - setDefault(handleInvalid, VectorIndexer.ERROR_INVALID) - /** * Threshold for the number of values a categorical feature can take. * If a feature is found to have {@literal >} maxCategories values, then it is declared @@ -75,10 +73,10 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu " If a feature is found to have > maxCategories values, then it is declared continuous.", ParamValidators.gtEq(2)) - setDefault(maxCategories -> 20) - /** @group getParam */ def getMaxCategories: Int = $(maxCategories) + + setDefault(maxCategories -> 20, handleInvalid -> VectorIndexer.ERROR_INVALID) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index be91844ba39e6..3590b9118f3b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -57,8 +57,6 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri "An array of indices to select features from a vector column." + " There can be no overlap with names.", VectorSlicer.validIndices) - setDefault(indices -> Array.emptyIntArray) - /** @group getParam */ @Since("1.5.0") def getIndices: Array[Int] = $(indices) @@ -79,8 +77,6 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri "An array of feature names to select features from a vector column." + " There can be no overlap with indices.", VectorSlicer.validNames) - setDefault(names -> Array.empty[String]) - /** @group getParam */ @Since("1.5.0") def getNames: Array[String] = $(names) @@ -97,6 +93,8 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) + setDefault(indices -> Array.emptyIntArray, names -> Array.empty[String]) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { // Validity checks diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index db2665fa2e4a3..01db39f9e3921 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -47,7 +47,6 @@ private[feature] trait Word2VecBase extends Params final val vectorSize = new IntParam( this, "vectorSize", "the dimension of codes after transforming from words (> 0)", ParamValidators.gt(0)) - setDefault(vectorSize -> 100) /** @group getParam */ def getVectorSize: Int = $(vectorSize) @@ -60,7 +59,6 @@ private[feature] trait Word2VecBase extends Params final val windowSize = new IntParam( this, "windowSize", "the window size (context words from [-window, window]) (> 0)", ParamValidators.gt(0)) - setDefault(windowSize -> 5) /** @group expertGetParam */ def getWindowSize: Int = $(windowSize) @@ -73,7 +71,6 @@ private[feature] trait Word2VecBase extends Params final val numPartitions = new IntParam( this, "numPartitions", "number of partitions for sentences of words (> 0)", ParamValidators.gt(0)) - setDefault(numPartitions -> 1) /** @group getParam */ def getNumPartitions: Int = $(numPartitions) @@ -86,7 +83,6 @@ private[feature] trait Word2VecBase extends Params */ final val minCount = new IntParam(this, "minCount", "the minimum number of times a token must " + "appear to be included in the word2vec model's vocabulary (>= 0)", ParamValidators.gtEq(0)) - setDefault(minCount -> 5) /** @group getParam */ def getMinCount: Int = $(minCount) @@ -101,13 +97,12 @@ private[feature] trait Word2VecBase extends Params final val maxSentenceLength = new IntParam(this, "maxSentenceLength", "Maximum length " + "(in words) of each sentence in the input data. Any sentence longer than this threshold will " + "be divided into chunks up to the size (> 0)", ParamValidators.gt(0)) - setDefault(maxSentenceLength -> 1000) /** @group getParam */ def getMaxSentenceLength: Int = $(maxSentenceLength) - setDefault(stepSize -> 0.025) - setDefault(maxIter -> 1) + setDefault(vectorSize -> 100, windowSize -> 5, numPartitions -> 1, minCount -> 5, + maxSentenceLength -> 1000, stepSize -> 0.025, maxIter -> 1) /** * Validate and transform the input schema. diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index f1a68edaed950..7aab4ef62c4d9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -50,7 +50,6 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { */ @Since("2.2.0") val itemsCol: Param[String] = new Param[String](this, "itemsCol", "items column name") - setDefault(itemsCol -> "items") /** @group getParam */ @Since("2.2.0") @@ -66,7 +65,6 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { val minSupport: DoubleParam = new DoubleParam(this, "minSupport", "the minimal support level of a frequent pattern", ParamValidators.inRange(0.0, 1.0)) - setDefault(minSupport -> 0.3) /** @group getParam */ @Since("2.2.0") @@ -95,12 +93,13 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { val minConfidence: DoubleParam = new DoubleParam(this, "minConfidence", "minimal confidence for generating Association Rule", ParamValidators.inRange(0.0, 1.0)) - setDefault(minConfidence -> 0.8) /** @group getParam */ @Since("2.2.0") def getMinConfidence: Double = $(minConfidence) + setDefault(minSupport -> 0.3, itemsCol -> "items", minConfidence -> 0.8) + /** * Validates and transforms the input schema. * @param schema input schema diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 2c30e44b93467..f301c349a2dc7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -62,7 +62,6 @@ private[regression] trait AFTSurvivalRegressionParams extends PredictorParams /** @group getParam */ @Since("1.6.0") def getCensorCol: String = $(censorCol) - setDefault(censorCol -> "censor") /** * Param for quantile probabilities array. @@ -78,7 +77,6 @@ private[regression] trait AFTSurvivalRegressionParams extends PredictorParams /** @group getParam */ @Since("1.6.0") def getQuantileProbabilities: Array[Double] = $(quantileProbabilities) - setDefault(quantileProbabilities -> Array(0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99)) /** * Param for quantiles column name. @@ -92,6 +90,10 @@ private[regression] trait AFTSurvivalRegressionParams extends PredictorParams @Since("1.6.0") def getQuantilesCol: String = $(quantilesCol) + setDefault(censorCol -> "censor", + quantileProbabilities -> Array(0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99), + fitIntercept -> true, maxIter -> 100, tol -> 1E-6, aggregationDepth -> 2, blockSize -> 1) + /** Checks whether the input has quantiles column name. */ private[regression] def hasQuantilesCol: Boolean = { isDefined(quantilesCol) && $(quantilesCol).nonEmpty @@ -153,7 +155,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("1.6.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Set the maximum number of iterations. @@ -162,7 +163,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("1.6.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the convergence tolerance of iterations. @@ -172,7 +172,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("1.6.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Suggested depth for treeAggregate (greater than or equal to 2). @@ -183,7 +182,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("2.1.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Set block size for stacking input data in matrices. @@ -202,7 +200,6 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) /** * Extract [[featuresCol]], [[labelCol]] and [[censorCol]] from input dataset, diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index de559142a9261..235a7f9b6ebd5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -105,6 +105,10 @@ private[regression] trait LinearRegressionParams extends PredictorParams @Since("2.3.0") def getEpsilon: Double = $(epsilon) + setDefault(regParam -> 0.0, fitIntercept -> true, standardization -> true, + elasticNetParam -> 0.0, maxIter -> 100, tol -> 1E-6, solver -> Auto, + aggregationDepth -> 2, loss -> SquaredError, epsilon -> 1.35, blockSize -> 1) + override protected def validateAndTransformSchema( schema: StructType, fitting: Boolean, @@ -191,7 +195,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.3.0") def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) /** * Set if we should fit the intercept. @@ -201,7 +204,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.5.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) /** * Whether to standardize the training features before fitting the model. @@ -217,7 +219,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.5.0") def setStandardization(value: Boolean): this.type = set(standardization, value) - setDefault(standardization -> true) /** * Set the ElasticNet mixing parameter. @@ -233,7 +234,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.4.0") def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) - setDefault(elasticNetParam -> 0.0) /** * Set the maximum number of iterations. @@ -243,7 +243,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.3.0") def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) /** * Set the convergence tolerance of iterations. @@ -254,7 +253,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.4.0") def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) /** * Whether to over-/under-sample training instances according to the given weights in weightCol. @@ -283,7 +281,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("1.6.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> Auto) /** * Suggested depth for treeAggregate (greater than or equal to 2). @@ -295,7 +292,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("2.1.0") def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) - setDefault(aggregationDepth -> 2) /** * Sets the value of param [[loss]]. @@ -305,7 +301,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("2.3.0") def setLoss(value: String): this.type = set(loss, value) - setDefault(loss -> SquaredError) /** * Sets the value of param [[epsilon]]. @@ -315,7 +310,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("2.3.0") def setEpsilon(value: Double): this.type = set(epsilon, value) - setDefault(epsilon -> 1.35) /** * Set block size for stacking input data in matrices. @@ -334,7 +328,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String */ @Since("3.1.0") def setBlockSize(value: Int): this.type = set(blockSize, value) - setDefault(blockSize -> 1) override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => instr.logPipelineStage(this) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index cc21e9cbbe8de..dfa4961d9ffb8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -174,22 +174,32 @@ class InternalNode private[ml] ( } override private[ml] def predictImpl(features: Vector): LeafNode = { - if (split.shouldGoLeft(features)) { - leftChild.predictImpl(features) - } else { - rightChild.predictImpl(features) + var node: Node = this + while (node.isInstanceOf[InternalNode]) { + val n = node.asInstanceOf[InternalNode] + if (n.split.shouldGoLeft(features)) { + node = n.leftChild + } else { + node = n.rightChild + } } + node.asInstanceOf[LeafNode] } override private[ml] def predictBinned( binned: Array[Int], splits: Array[Array[Split]]): LeafNode = { - val i = split.featureIndex - if (split.shouldGoLeft(binned(i), splits(i))) { - leftChild.predictBinned(binned, splits) - } else { - rightChild.predictBinned(binned, splits) + var node: Node = this + while (node.isInstanceOf[InternalNode]) { + val n = node.asInstanceOf[InternalNode] + val i = n.split.featureIndex + if (n.split.shouldGoLeft(binned(i), splits(i))) { + node = n.leftChild + } else { + node = n.rightChild + } } + node.asInstanceOf[LeafNode] } override private[tree] def numDescendants: Int = { @@ -326,27 +336,27 @@ private[tree] class LearningNode( * [[org.apache.spark.ml.tree.impl.RandomForest.findBestSplits()]]. */ def predictImpl(binnedFeatures: Array[Int], splits: Array[Array[Split]]): Int = { - if (this.isLeaf || this.split.isEmpty) { - this.id - } else { - val split = this.split.get + var node = this + while (!node.isLeaf && node.split.nonEmpty) { + val split = node.split.get val featureIndex = split.featureIndex val splitLeft = split.shouldGoLeft(binnedFeatures(featureIndex), splits(featureIndex)) - if (this.leftChild.isEmpty) { + if (node.leftChild.isEmpty) { // Not yet split. Return next layer of nodes to train if (splitLeft) { - LearningNode.leftChildIndex(this.id) + return LearningNode.leftChildIndex(node.id) } else { - LearningNode.rightChildIndex(this.id) + return LearningNode.rightChildIndex(node.id) } } else { if (splitLeft) { - this.leftChild.get.predictImpl(binnedFeatures, splits) + node = node.leftChild.get } else { - this.rightChild.get.predictImpl(binnedFeatures, splits) + node = node.rightChild.get } } } + node.id } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 7e2c287f146fb..19ea8ae4775d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -333,8 +333,6 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { "Fraction of the training data used for learning each decision tree, in range (0, 1].", ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) - setDefault(subsamplingRate -> 1.0) - /** @group getParam */ final def getSubsamplingRate: Double = $(subsamplingRate) @@ -386,10 +384,10 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { || Try(value.toInt).filter(_ > 0).isSuccess || Try(value.toDouble).filter(_ > 0).filter(_ <= 1.0).isSuccess) - setDefault(featureSubsetStrategy -> "auto") - /** @group getParam */ final def getFeatureSubsetStrategy: String = $(featureSubsetStrategy).toLowerCase(Locale.ROOT) + + setDefault(subsamplingRate -> 1.0, featureSubsetStrategy -> "auto") } /** @@ -448,8 +446,6 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { new IntParam(this, "numTrees", "Number of trees to train (at least 1)", ParamValidators.gtEq(1)) - setDefault(numTrees -> 20) - /** @group getParam */ final def getNumTrees: Int = $(numTrees) @@ -461,11 +457,11 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { final val bootstrap: BooleanParam = new BooleanParam(this, "bootstrap", "Whether bootstrap samples are used when building trees.") - setDefault(bootstrap -> true) - /** @group getParam */ @Since("3.0.0") final def getBootstrap: Boolean = $(bootstrap) + + setDefault(numTrees -> 20, bootstrap -> true) } private[ml] trait RandomForestClassifierParams @@ -518,9 +514,7 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasS "(a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each estimator.", ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) - setDefault(maxIter -> 20, stepSize -> 0.1, validationTol -> 0.01) - - setDefault(featureSubsetStrategy -> "all") + setDefault(maxIter -> 20, stepSize -> 0.1, validationTol -> 0.01, featureSubsetStrategy -> "all") /** (private[ml]) Create a BoostingStrategy instance to use with the old API. */ private[ml] def getOldBoostingStrategy( diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index e99c55b0cdd85..275d3c5510f7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -55,8 +55,6 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { /** @group getParam */ def getNumFolds: Int = $(numFolds) - setDefault(numFolds -> 3) - /** * Param for the column name of user specified fold number. Once this is specified, * `CrossValidator` won't do random k-fold split. Note that this column should be @@ -68,7 +66,7 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { def getFoldCol: String = $(foldCol) - setDefault(foldCol, "") + setDefault(foldCol -> "", numFolds -> 3) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index d177364d012f4..0ac8cd014c6e4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -534,8 +534,10 @@ object MLUtils extends Logging { norm2: Double, precision: Double = 1e-6): Double = { val n = v1.size - require(v2.size == n) - require(norm1 >= 0.0 && norm2 >= 0.0) + require(v2.size == n, + s"Both vectors should have same length, found v1 is $n while v2 is ${v2.size}") + require(norm1 >= 0.0 && norm2 >= 0.0, + s"Both norms should be greater or equal to 0.0, found norm1=$norm1, norm2=$norm2") var sqDist = 0.0 /* * The relative error is diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala index 35586320cb82b..fdd6e352fa639 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala @@ -45,7 +45,7 @@ class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { val trainer = new FeedForwardTrainer(topology, 2, 1) trainer.setWeights(initialWeights) trainer.LBFGSOptimizer.setNumIterations(20) - val model = trainer.train(rddData) + val (model, _) = trainer.train(rddData) val predictionAndLabels = rddData.map { case (input, label) => (model.predict(input)(0), label(0)) }.collect() @@ -80,7 +80,7 @@ class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { // TODO: add a test for SGD trainer.LBFGSOptimizer.setConvergenceTol(1e-4).setNumIterations(20) trainer.setWeights(initialWeights).setStackSize(1) - val model = trainer.train(rddData) + val (model, _) = trainer.train(rddData) val predictionAndLabels = rddData.map { case (input, label) => (model.predict(input), label) }.collect() diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index 902af71e42f86..c909e72c689bc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -242,4 +242,36 @@ class MultilayerPerceptronClassifierSuite extends MLTest with DefaultReadWriteTe val sparkVersionStr = metadata.select("sparkVersion").first().getString(0) assert(sparkVersionStr == "2.4.4") } + + test("summary and training summary") { + val mlp = new MultilayerPerceptronClassifier() + val model = mlp.setMaxIter(5).setLayers(Array(2, 3, 2)).fit(dataset) + val summary = model.evaluate(dataset) + + assert(model.summary.truePositiveRateByLabel === summary.truePositiveRateByLabel) + assert(model.summary.falsePositiveRateByLabel === summary.falsePositiveRateByLabel) + assert(model.summary.precisionByLabel === summary.precisionByLabel) + assert(model.summary.recallByLabel === summary.recallByLabel) + assert(model.summary.fMeasureByLabel === summary.fMeasureByLabel) + assert(model.summary.accuracy === summary.accuracy) + assert(model.summary.weightedFalsePositiveRate === summary.weightedFalsePositiveRate) + assert(model.summary.weightedTruePositiveRate === summary.weightedTruePositiveRate) + assert(model.summary.weightedPrecision === summary.weightedPrecision) + assert(model.summary.weightedRecall === summary.weightedRecall) + assert(model.summary.weightedFMeasure === summary.weightedFMeasure) + } + + test("MultilayerPerceptron training summary totalIterations") { + Seq(1, 5, 10, 20, 100).foreach { maxIter => + val trainer = new MultilayerPerceptronClassifier() + .setMaxIter(maxIter) + .setLayers(Array(2, 3, 2)) + val model = trainer.fit(dataset) + if (maxIter == 1) { + assert(model.summary.totalIterations === maxIter) + } else { + assert(model.summary.totalIterations <= maxIter) + } + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala index 4d9e664850c12..dd0139b94f098 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala @@ -63,6 +63,9 @@ trait DefaultReadWriteTest extends TempDirectory { self: Suite => (instance.getOrDefault(p), newInstance.getOrDefault(p)) match { case (Array(values), Array(newValues)) => assert(values === newValues, s"Values do not match on param ${p.name}.") + case (value: Double, newValue: Double) => + assert(value.isNaN && newValue.isNaN || value == newValue, + s"Values do not match on param ${p.name}.") case (value, newValue) => assert(value === newValue, s"Values do not match on param ${p.name}.") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index d96a4da46a630..584e7555eb0d9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -21,8 +21,8 @@ import scala.collection.JavaConverters._ import scala.util.Random import scala.util.control.Breaks._ -import org.scalatest.Assertions._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.{Vector, Vectors} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index a5542565c7401..edea67e524aad 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.optimization import scala.collection.JavaConverters._ import scala.util.Random -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index 69c303ee932e0..27e21acc275c3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.optimization import scala.util.Random -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala index 02ea74b87f684..8066900dfa011 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.regression -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext diff --git a/pom.xml b/pom.xml index cfcb55b27fa99..6b72e7d13643b 100644 --- a/pom.xml +++ b/pom.xml @@ -116,6 +116,7 @@ ${java.version} ${java.version} 3.6.3 + 1.6.0 spark 1.7.30 1.2.17 @@ -162,6 +163,7 @@ 3.2.2 2.12.10 2.12 + 2.0.0 --test true @@ -249,6 +251,8 @@ 1g + + yyyy-MM-dd HH:mm:ss z @@ -334,6 +338,21 @@ scalatest_${scala.binary.version} test + + org.scalatestplus + scalatestplus-scalacheck_${scala.binary.version} + test + + + org.scalatestplus + scalatestplus-mockito_${scala.binary.version} + test + + + org.scalatestplus + scalatestplus-selenium_${scala.binary.version} + test + junit junit @@ -697,7 +716,7 @@ org.roaringbitmap RoaringBitmap - 0.7.45 + 0.9.0 commons-net @@ -707,7 +726,7 @@ io.netty netty-all - 4.1.47.Final + 4.1.51.Final org.apache.derby @@ -846,7 +865,7 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.6.6 + 3.7.0-M5 com.fasterxml.jackson.core @@ -892,7 +911,25 @@ org.scalatest scalatest_${scala.binary.version} - 3.0.8 + 3.2.0 + test + + + org.scalatestplus + scalatestplus-scalacheck_${scala.binary.version} + 3.1.0.0-RC2 + test + + + org.scalatestplus + scalatestplus-mockito_${scala.binary.version} + 1.0.0-SNAP5 + test + + + org.scalatestplus + scalatestplus-selenium_${scala.binary.version} + 1.0.0-SNAP5 test @@ -2370,11 +2407,39 @@ - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - + + org.codehaus.mojo + build-helper-maven-plugin + 3.2.0 + + + module-timestamp-property + validate + + timestamp-property + + + module.build.timestamp + ${maven.build.timestamp.format} + current + America/Los_Angeles + + + + local-timestamp-property + validate + + timestamp-property + + + local.build.timestamp + ${maven.build.timestamp.format} + build + America/Los_Angeles + + + + net.alchim31.maven scala-maven-plugin @@ -2507,7 +2572,7 @@ org.scalatest scalatest-maven-plugin - 2.0.0 + ${scalatest-maven-plugin.version} ${project.build.directory}/surefire-reports @@ -2649,7 +2714,7 @@ org.codehaus.mojo exec-maven-plugin - 1.6.0 + ${exec-maven-plugin.version} org.apache.maven.plugins diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d19b514d662fa..5a3ac213c2057 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -469,7 +469,6 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite", "org.apache.spark.sql.hive.HiveExternalCatalogSuite", "org.apache.spark.sql.hive.StatisticsSuite", - "org.apache.spark.sql.hive.execution.HiveCompatibilitySuite", "org.apache.spark.sql.hive.client.VersionsSuite", "org.apache.spark.sql.hive.client.HiveClientVersions", "org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite", @@ -486,9 +485,14 @@ object SparkParallelTestGrouping { ) private val DEFAULT_TEST_GROUP = "default_test_group" + private val HIVE_EXECUTION_TEST_GROUP = "hive_execution_test_group" private def testNameToTestGroup(name: String): String = name match { case _ if testsWhichShouldRunInTheirOwnDedicatedJvm.contains(name) => name + // Different with the cases in testsWhichShouldRunInTheirOwnDedicatedJvm, here we are grouping + // all suites of `org.apache.spark.sql.hive.execution.*` into a single group, instead of + // launching one JVM per suite. + case _ if name.contains("org.apache.spark.sql.hive.execution") => HIVE_EXECUTION_TEST_GROUP case _ => DEFAULT_TEST_GROUP } @@ -1028,8 +1032,6 @@ object TestSettings { // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), - // Required to detect Junit tests for each project, see also https://github.com/sbt/junit-interface/issues/35 - crossPaths := false, // Enable Junit testing. libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test", // `parallelExecutionInTest` controls whether test suites belonging to the same SBT project diff --git a/python/docs/Makefile b/python/docs/Makefile index 4272b7488d9a0..763f493a0eb58 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -3,8 +3,8 @@ # You can set these variables from the command line. SPHINXOPTS ?= SPHINXBUILD ?= sphinx-build -SOURCEDIR ?= . -BUILDDIR ?= _build +SOURCEDIR ?= source +BUILDDIR ?= build export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.9-src.zip) diff --git a/python/docs/_static/pyspark.css b/python/docs/_static/pyspark.css deleted file mode 100644 index 41106f2f6e26d..0000000000000 --- a/python/docs/_static/pyspark.css +++ /dev/null @@ -1,90 +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. -*/ - -body { - background-color: #ffffff; -} - -div.sphinxsidebar { - width: 274px; -} - -div.bodywrapper { - margin: 0 0 0 274px; -} - -div.sphinxsidebar ul { - margin-right: 10px; -} - -div.sphinxsidebar li a { - word-break: break-all; -} - -span.pys-tag { - font-size: 11px; - font-weight: bold; - margin: 0 0 0 2px; - padding: 1px 3px 1px 3px; - -moz-border-radius: 3px; - -webkit-border-radius: 3px; - border-radius: 3px; - text-align: center; - text-decoration: none; -} - -span.pys-tag-experimental { - background-color: rgb(37, 112, 128); - color: rgb(255, 255, 255); -} - -span.pys-tag-deprecated { - background-color: rgb(238, 238, 238); - color: rgb(62, 67, 73); -} - -div.pys-note-experimental { - background-color: rgb(88, 151, 165); - border-color: rgb(59, 115, 127); - color: rgb(255, 255, 255); -} - -div.pys-note-deprecated { -} - -.hasTooltip { - position:relative; -} -.hasTooltip span { - display:none; -} - -.hasTooltip:hover span.tooltip { - display: inline-block; - -moz-border-radius: 2px; - -webkit-border-radius: 2px; - border-radius: 2px; - background-color: rgb(250, 250, 250); - color: rgb(68, 68, 68); - font-weight: normal; - box-shadow: 1px 1px 3px rgb(127, 127, 127); - position: absolute; - padding: 0 3px 0 3px; - top: 1.3em; - left: 14px; - z-index: 9999 -} diff --git a/python/docs/_static/pyspark.js b/python/docs/_static/pyspark.js deleted file mode 100644 index 75e4c42492a48..0000000000000 --- a/python/docs/_static/pyspark.js +++ /dev/null @@ -1,99 +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. -*/ - -$(function (){ - - function startsWith(s, prefix) { - return s && s.indexOf(prefix) === 0; - } - - function buildSidebarLinkMap() { - var linkMap = {}; - $('div.sphinxsidebar a.reference.internal').each(function (i,a) { - var href = $(a).attr('href'); - if (startsWith(href, '#module-')) { - var id = href.substr(8); - linkMap[id] = [$(a), null]; - } - }) - return linkMap; - }; - - function getAdNoteDivs(dd) { - var noteDivs = {}; - dd.find('> div.admonition.note > p.last').each(function (i, p) { - var text = $(p).text(); - if (!noteDivs.experimental && startsWith(text, 'Experimental')) { - noteDivs.experimental = $(p).parent(); - } - if (!noteDivs.deprecated && startsWith(text, 'Deprecated')) { - noteDivs.deprecated = $(p).parent(); - } - }); - return noteDivs; - } - - function getParentId(name) { - var last_idx = name.lastIndexOf('.'); - return last_idx == -1? '': name.substr(0, last_idx); - } - - function buildTag(text, cls, tooltip) { - return '' + text + '' - + tooltip + '' - } - - - var sidebarLinkMap = buildSidebarLinkMap(); - - $('dl.class, dl.function').each(function (i,dl) { - - dl = $(dl); - dt = dl.children('dt').eq(0); - dd = dl.children('dd').eq(0); - var id = dt.attr('id'); - var desc = dt.find('> .descname').text(); - var adNoteDivs = getAdNoteDivs(dd); - - if (id) { - var parent_id = getParentId(id); - - var r = sidebarLinkMap[parent_id]; - if (r) { - if (r[1] === null) { - r[1] = $('
    '); - r[0].parent().append(r[1]); - } - var tags = ''; - if (adNoteDivs.experimental) { - tags += buildTag('E', 'pys-tag-experimental', 'Experimental'); - adNoteDivs.experimental.addClass('pys-note pys-note-experimental'); - } - if (adNoteDivs.deprecated) { - tags += buildTag('D', 'pys-tag-deprecated', 'Deprecated'); - adNoteDivs.deprecated.addClass('pys-note pys-note-deprecated'); - } - var li = $('
  • '); - var a = $('' + desc + ''); - li.append(a); - li.append(tags); - r[1].append(li); - sidebarLinkMap[id] = [a, null]; - } - } - }); -}); diff --git a/python/docs/_templates/layout.html b/python/docs/_templates/layout.html deleted file mode 100644 index ab36ebababf88..0000000000000 --- a/python/docs/_templates/layout.html +++ /dev/null @@ -1,6 +0,0 @@ -{% extends "!layout.html" %} -{% set script_files = script_files + ["_static/pyspark.js"] %} -{% set css_files = css_files + ['_static/pyspark.css'] %} -{% block rootrellink %} - {{ super() }} -{% endblock %} diff --git a/python/docs/index.rst b/python/docs/index.rst deleted file mode 100644 index 6e059264e6bbb..0000000000000 --- a/python/docs/index.rst +++ /dev/null @@ -1,53 +0,0 @@ -.. pyspark documentation master file, created by - sphinx-quickstart on Thu Aug 28 15:17:47 2014. - You can adapt this file completely to your liking, but it should at least - contain the root `toctree` directive. - -Welcome to Spark Python API Docs! -=================================== - -Contents: - -.. toctree:: - :maxdepth: 2 - - pyspark - pyspark.sql - pyspark.streaming - pyspark.ml - pyspark.mllib - pyspark.resource - - -Core classes: ---------------- - - :class:`pyspark.SparkContext` - - Main entry point for Spark functionality. - - :class:`pyspark.RDD` - - A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. - - :class:`pyspark.streaming.StreamingContext` - - Main entry point for Spark Streaming functionality. - - :class:`pyspark.streaming.DStream` - - A Discretized Stream (DStream), the basic abstraction in Spark Streaming. - - :class:`pyspark.sql.SparkSession` - - Main entry point for DataFrame and SQL functionality. - - :class:`pyspark.sql.DataFrame` - - A distributed collection of data grouped into named columns. - - -Indices and tables -================== - -* :ref:`search` diff --git a/python/docs/make2.bat b/python/docs/make2.bat index 7955a83051b8e..2f87032820f42 100644 --- a/python/docs/make2.bat +++ b/python/docs/make2.bat @@ -5,8 +5,8 @@ REM Command file for Sphinx documentation if "%SPHINXBUILD%" == "" ( set SPHINXBUILD=sphinx-build ) -set SOURCEDIR=. -set BUILDDIR=_build +set SOURCEDIR=source +set BUILDDIR=build set PYTHONPATH=..;..\lib\py4j-0.10.9-src.zip diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst deleted file mode 100644 index e31dfddd5988e..0000000000000 --- a/python/docs/pyspark.ml.rst +++ /dev/null @@ -1,122 +0,0 @@ -pyspark.ml package -================== - -ML Pipeline APIs ----------------- - -.. automodule:: pyspark.ml - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.param module ------------------------ - -.. automodule:: pyspark.ml.param - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.feature module -------------------------- - -.. automodule:: pyspark.ml.feature - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.classification module --------------------------------- - -.. automodule:: pyspark.ml.classification - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.clustering module ----------------------------- - -.. automodule:: pyspark.ml.clustering - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.functions module ----------------------------- - -.. automodule:: pyspark.ml.functions - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.linalg module ----------------------------- - -.. automodule:: pyspark.ml.linalg - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.recommendation module --------------------------------- - -.. automodule:: pyspark.ml.recommendation - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.regression module ----------------------------- - -.. automodule:: pyspark.ml.regression - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.stat module ----------------------- - -.. automodule:: pyspark.ml.stat - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.tuning module ------------------------- - -.. automodule:: pyspark.ml.tuning - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.evaluation module ----------------------------- - -.. automodule:: pyspark.ml.evaluation - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.fpm module ----------------------------- - -.. automodule:: pyspark.ml.fpm - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.image module ----------------------------- - -.. automodule:: pyspark.ml.image - :members: - :undoc-members: - :inherited-members: - -pyspark.ml.util module ----------------------------- - -.. automodule:: pyspark.ml.util - :members: - :undoc-members: - :inherited-members: diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst deleted file mode 100644 index 2d54ab118b94b..0000000000000 --- a/python/docs/pyspark.mllib.rst +++ /dev/null @@ -1,99 +0,0 @@ -pyspark.mllib package -===================== - -pyspark.mllib.classification module ------------------------------------ - -.. automodule:: pyspark.mllib.classification - :members: - :undoc-members: - :inherited-members: - -pyspark.mllib.clustering module -------------------------------- - -.. automodule:: pyspark.mllib.clustering - :members: - :undoc-members: - -pyspark.mllib.evaluation module -------------------------------- - -.. automodule:: pyspark.mllib.evaluation - :members: - :undoc-members: - -pyspark.mllib.feature module -------------------------------- - -.. automodule:: pyspark.mllib.feature - :members: - :undoc-members: - :show-inheritance: - -pyspark.mllib.fpm module ------------------------- - -.. automodule:: pyspark.mllib.fpm - :members: - :undoc-members: - -pyspark.mllib.linalg module ---------------------------- - -.. automodule:: pyspark.mllib.linalg - :members: - :undoc-members: - :show-inheritance: - -pyspark.mllib.linalg.distributed module ---------------------------------------- - -.. automodule:: pyspark.mllib.linalg.distributed - :members: - :undoc-members: - :show-inheritance: - -pyspark.mllib.random module ---------------------------- - -.. automodule:: pyspark.mllib.random - :members: - :undoc-members: - -pyspark.mllib.recommendation module ------------------------------------ - -.. automodule:: pyspark.mllib.recommendation - :members: - :undoc-members: - -pyspark.mllib.regression module -------------------------------- - -.. automodule:: pyspark.mllib.regression - :members: - :undoc-members: - :inherited-members: - -pyspark.mllib.stat module -------------------------- - -.. automodule:: pyspark.mllib.stat - :members: - :undoc-members: - -pyspark.mllib.tree module -------------------------- - -.. automodule:: pyspark.mllib.tree - :members: - :undoc-members: - :inherited-members: - -pyspark.mllib.util module -------------------------- - -.. automodule:: pyspark.mllib.util - :members: - :undoc-members: diff --git a/python/docs/pyspark.resource.rst b/python/docs/pyspark.resource.rst deleted file mode 100644 index 7f3a79b9e5b52..0000000000000 --- a/python/docs/pyspark.resource.rst +++ /dev/null @@ -1,11 +0,0 @@ -pyspark.resource module -======================= - -Module Contents ---------------- - -.. automodule:: pyspark.resource - :members: - :undoc-members: - :inherited-members: - diff --git a/python/docs/pyspark.rst b/python/docs/pyspark.rst deleted file mode 100644 index 402d6ce9eb016..0000000000000 --- a/python/docs/pyspark.rst +++ /dev/null @@ -1,21 +0,0 @@ -pyspark package -=============== - -Subpackages ------------ - -.. toctree:: - :maxdepth: 1 - - pyspark.sql - pyspark.streaming - pyspark.ml - pyspark.mllib - pyspark.resource - -Contents --------- - -.. automodule:: pyspark - :members: - :undoc-members: diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst deleted file mode 100644 index 406ada701941a..0000000000000 --- a/python/docs/pyspark.sql.rst +++ /dev/null @@ -1,37 +0,0 @@ -pyspark.sql module -================== - -Module Contents ---------------- - -.. automodule:: pyspark.sql - :members: - :undoc-members: - :inherited-members: - :exclude-members: builder -.. We need `exclude-members` to prevent default description generations - as a workaround for old Sphinx (< 1.6.6). - -pyspark.sql.types module ------------------------- -.. automodule:: pyspark.sql.types - :members: - :undoc-members: - -pyspark.sql.functions module ----------------------------- -.. automodule:: pyspark.sql.functions - :members: - :undoc-members: - -pyspark.sql.avro.functions module ---------------------------------- -.. automodule:: pyspark.sql.avro.functions - :members: - :undoc-members: - -pyspark.sql.streaming module ----------------------------- -.. automodule:: pyspark.sql.streaming - :members: - :undoc-members: diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst deleted file mode 100644 index f7df6438b9169..0000000000000 --- a/python/docs/pyspark.streaming.rst +++ /dev/null @@ -1,17 +0,0 @@ -pyspark.streaming module -======================== - -Module contents ---------------- - -.. automodule:: pyspark.streaming - :members: - :undoc-members: - :show-inheritance: - -pyspark.streaming.kinesis module --------------------------------- -.. automodule:: pyspark.streaming.kinesis - :members: - :undoc-members: - :show-inheritance: diff --git a/python/docs/_static/copybutton.js b/python/docs/source/_static/copybutton.js similarity index 100% rename from python/docs/_static/copybutton.js rename to python/docs/source/_static/copybutton.js diff --git a/python/docs/source/_static/css/pyspark.css b/python/docs/source/_static/css/pyspark.css new file mode 100644 index 0000000000000..2fd8720e2fa0d --- /dev/null +++ b/python/docs/source/_static/css/pyspark.css @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/* PySpark style CSS overwrite */ + +/* Lato font (from its parent) does not make any difference when it's bold. Defaults to 'Source Sans Pro' */ +body { + font-family:"Source Sans Pro",sans-serif!important; +} + +h1,h2 { + color:#1B5162!important; +} + +h3 { + color: #555555 +} + +/* Top menu */ +#navbar-main { + background: #1B5162!important; + box-shadow: 0px 2px 4px rgba(0, 0, 0, 0.11); +} + +#navbar-main-elements li.nav-item a { + color: rgba(255, 255, 255, 0.8); +} + +#navbar-main-elements li.active a { + font-weight: 600; + color: #FFFFFF!important; +} + +.col-9 { + flex: 0 0 80%; + max-width: 80%; +} + +/* Left pannel size */ +@media (min-width: 768px) { + .col-md-3 { + flex: 0 0 20%; + max-width: 20%; + } +} + +/* Top menu right button */ +.navbar-toggler { + color:rgba(255,255,255,.5)!important; + border-color:rgba(255,255,255,.5)!important; +} + +.navbar-toggler-icon { + background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(255,255,255,.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")!important; +} + +/* Left bar list colors */ +.bd-sidebar .nav>.active>a { + color: #1B5162!important; +} + +.bd-sidebar .nav>li>a:hover { + color: #1B5162!important; +} + +.bd-sidebar .nav>.active:hover>a,.bd-sidebar .nav>.active>a { + color: #1B5162!important; +} + +u.bd-sidebar .nav>li>ul>.active:hover>a,.bd-sidebar .nav>li>ul>.active>a { + color: #1B5162!important; +} + +/* Right bar list colors */ +.toc-entry>.nav-link.active { + color: #1B5162!important; + border-left: 2px solid #1B5162!important; +} + diff --git a/python/docs/source/_templates/class_with_docs.rst b/python/docs/source/_templates/class_with_docs.rst new file mode 100644 index 0000000000000..7c37b83c0e90e --- /dev/null +++ b/python/docs/source/_templates/class_with_docs.rst @@ -0,0 +1,79 @@ +.. 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. + + +{{ objname }} +{{ underline }} + +.. currentmodule:: {{ module }} + +.. autoclass:: {{ objname }} + + {% if '__init__' in methods %} + {% set caught_result = methods.remove('__init__') %} + {% endif %} + + {% block methods_summary %} + {% if methods %} + + .. rubric:: Methods + + .. autosummary:: + {% for item in methods %} + ~{{ name }}.{{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + + {% block attributes_summary %} + {% if attributes %} + + .. rubric:: Attributes + + .. autosummary:: + {% for item in attributes %} + ~{{ name }}.{{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + + {% block methods_documentation %} + {% if methods %} + + .. rubric:: Methods Documentation + + {% for item in methods %} + .. automethod:: {{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + + {% block attributes_documentation %} + {% if attributes %} + + .. rubric:: Attributes Documentation + + {% for item in attributes %} + .. autoattribute:: {{ item }} + {%- endfor %} + + {% endif %} + {% endblock %} + diff --git a/python/docs/conf.py b/python/docs/source/conf.py similarity index 89% rename from python/docs/conf.py rename to python/docs/source/conf.py index 9e7afb7c07298..7b1939d976080 100644 --- a/python/docs/conf.py +++ b/python/docs/source/conf.py @@ -14,12 +14,24 @@ import sys import os +import shutil +import errno # If extensions (or modules to document with autodoc) are in another directory, # add these directories to sys.path here. If the directory is relative to the # documentation root, use os.path.abspath to make it absolute, like shown here. sys.path.insert(0, os.path.abspath('.')) +# Remove previously generated rst files. Ignore errors just in case it stops +# generating whole docs. +shutil.rmtree( + "%s/reference/api" % os.path.dirname(os.path.abspath(__file__)), ignore_errors=True) +try: + os.mkdir("%s/reference/api" % os.path.dirname(os.path.abspath(__file__))) +except OSError as e: + if e.errno != errno.EEXIST: + raise + # -- General configuration ------------------------------------------------ # If your documentation needs a minimal Sphinx version, state it here. @@ -32,6 +44,7 @@ 'sphinx.ext.autodoc', 'sphinx.ext.viewcode', 'sphinx.ext.mathjax', + 'sphinx.ext.autosummary', ] # Add any paths that contain templates here, relative to this directory. @@ -47,8 +60,8 @@ master_doc = 'index' # General information about the project. -project = u'PySpark' -copyright = u'' +project = 'PySpark' +copyright = '' # The version info for the project you're documenting, acts as replacement for # |version| and |release|, also used in various other places throughout the @@ -101,12 +114,13 @@ # Look at the first line of the docstring for function and method signatures. autodoc_docstring_signature = True +autosummary_generate = True # -- Options for HTML output ---------------------------------------------- # The theme to use for HTML and HTML Help pages. See the documentation for # a list of builtin themes. -html_theme = 'nature' +html_theme = 'pydata_sphinx_theme' # Theme options are theme-specific and customize the look and feel of a theme # further. For a list of options available for each theme, see the @@ -125,7 +139,7 @@ # The name of an image file (relative to this directory) to place at the top # of the sidebar. -html_logo = "../../docs/img/spark-logo-hd.png" +html_logo = "../../../docs/img/spark-logo-reverse.png" # The name of an image file (within the static path) to use as favicon of the # docs. This file should be a Windows icon file (.ico) being 16x16 or 32x32 @@ -137,6 +151,10 @@ # so a file named "default.css" will overwrite the builtin "default.css". html_static_path = ['_static'] +html_css_files = [ + 'css/pyspark.css', +] + # Add any extra paths that contain custom files (such as robots.txt or # .htaccess) here, relative to this directory. These files are copied # directly to the root of the documentation. @@ -204,8 +222,8 @@ # (source start file, target name, title, # author, documentclass [howto, manual, or own class]). latex_documents = [ - ('index', 'pyspark.tex', u'pyspark Documentation', - u'Author', 'manual'), + ('index', 'pyspark.tex', 'pyspark Documentation', + 'Author', 'manual'), ] # The name of an image file (relative to this directory) to place at the top of @@ -234,8 +252,8 @@ # One entry per manual page. List of tuples # (source start file, name, description, authors, manual section). man_pages = [ - ('index', 'pyspark', u'pyspark Documentation', - [u'Author'], 1) + ('index', 'pyspark', 'pyspark Documentation', + ['Author'], 1) ] # If true, show URL addresses after external links. @@ -248,8 +266,8 @@ # (source start file, target name, title, author, # dir menu entry, description, category) texinfo_documents = [ - ('index', 'pyspark', u'pyspark Documentation', - u'Author', 'pyspark', 'One line description of project.', + ('index', 'pyspark', 'pyspark Documentation', + 'Author', 'pyspark', 'One line description of project.', 'Miscellaneous'), ] @@ -269,13 +287,13 @@ # -- Options for Epub output ---------------------------------------------- # Bibliographic Dublin Core info. -epub_title = u'pyspark' -epub_author = u'Author' -epub_publisher = u'Author' -epub_copyright = u'2014, Author' +epub_title = 'pyspark' +epub_author = 'Author' +epub_publisher = 'Author' +epub_copyright = '2014, Author' # The basename for the epub file. It defaults to the project name. -#epub_basename = u'pyspark' +#epub_basename = 'pyspark' # The HTML theme for the epub output. Since the default themes are not optimized # for small screen space, using the same theme for HTML and epub output is @@ -335,7 +353,8 @@ # If false, no index is generated. #epub_use_index = True def setup(app): - app.add_javascript('copybutton.js') + # The app.add_javascript() is deprecated. + getattr(app, "add_js_file", getattr(app, "add_javascript"))('copybutton.js') # Skip sample endpoint link (not expected to resolve) linkcheck_ignore = [r'https://kinesis.us-east-1.amazonaws.com'] diff --git a/python/docs/source/development/index.rst b/python/docs/source/development/index.rst new file mode 100644 index 0000000000000..db9f667332635 --- /dev/null +++ b/python/docs/source/development/index.rst @@ -0,0 +1,21 @@ +.. 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. + +=========== +Development +=========== + diff --git a/python/docs/source/getting_started/index.rst b/python/docs/source/getting_started/index.rst new file mode 100644 index 0000000000000..457368c8194cb --- /dev/null +++ b/python/docs/source/getting_started/index.rst @@ -0,0 +1,22 @@ +.. 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. + + +=============== +Getting Started +=============== + diff --git a/python/docs/source/index.rst b/python/docs/source/index.rst new file mode 100644 index 0000000000000..b9180cefe5dcc --- /dev/null +++ b/python/docs/source/index.rst @@ -0,0 +1,68 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. PySpark documentation master file + +===================== +PySpark Documentation +===================== + +.. TODO(SPARK-32204): Add Binder integration at Live Notebook. + +PySpark is an interface for Apache Spark in Python. It not only allows you to write +Spark applications using Python APIs, but also provides the PySpark shell for +interactively analyzing your data in a distributed environment. PySpark supports most +of Spark's features such as Spark SQL, DataFrame, Streaming, MLlib +(Machine Learning) and Spark Core. + +.. image:: ../../../docs/img/pyspark-components.png + :alt: PySpark Compoenents + +**Spark SQL and DataFrame** + +Spark SQL is a Spark module for structured data processing. It provides +a programming abstraction called DataFrame and can also act as distributed +SQL query engine. + +**Streaming** + +Running on top of Spark, the streaming feature in Apache Spark enables powerful +interactive and analytical applications across both streaming and historical data, +while inheriting Spark’s ease of use and fault tolerance characteristics. + +**MLlib** + +Built on top of Spark, MLlib is a scalable machine learning library that provides +a uniform set of high-level APIs that help users create and tune practical machine +learning pipelines. + +**Spark Core** + +Spark Core is the underlying general execution engine for the Spark platform that all +other functionality is built on top of. It provides an RDD (Resilient Distributed Dataset) +and in-memory computing capabilities. + +.. toctree:: + :maxdepth: 2 + :hidden: + + getting_started/index + user_guide/index + reference/index + development/index + migration_guide/index + diff --git a/python/docs/source/migration_guide/index.rst b/python/docs/source/migration_guide/index.rst new file mode 100644 index 0000000000000..fc12668f81a58 --- /dev/null +++ b/python/docs/source/migration_guide/index.rst @@ -0,0 +1,22 @@ +.. 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. + + +=============== +Migration Guide +=============== + diff --git a/python/docs/source/reference/index.rst b/python/docs/source/reference/index.rst new file mode 100644 index 0000000000000..77f17da720dd5 --- /dev/null +++ b/python/docs/source/reference/index.rst @@ -0,0 +1,34 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + +============= +API Reference +============= + +This page lists an overview of all public PySpark modules, classes, functions and methods. + +.. toctree:: + :maxdepth: 2 + + pyspark.sql + pyspark.ss + pyspark.ml + pyspark.streaming + pyspark.mllib + pyspark + pyspark.resource diff --git a/python/docs/source/reference/pyspark.ml.rst b/python/docs/source/reference/pyspark.ml.rst new file mode 100644 index 0000000000000..00ee7b9078329 --- /dev/null +++ b/python/docs/source/reference/pyspark.ml.rst @@ -0,0 +1,365 @@ +.. 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. + + +ML +== + +ML Pipeline APIs +---------------- + +.. currentmodule:: pyspark.ml + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Transformer + UnaryTransformer + Estimator + Model + Predictor + PredictionModel + Pipeline + PipelineModel + + +Parameters +---------- + +.. currentmodule:: pyspark.ml.param + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Param + Params + TypeConverters + + +Feature +------- + +.. currentmodule:: pyspark.ml.feature + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ANOVASelector + ANOVASelectorModel + Binarizer + BucketedRandomProjectionLSH + BucketedRandomProjectionLSHModel + Bucketizer + ChiSqSelector + ChiSqSelectorModel + CountVectorizer + CountVectorizerModel + DCT + ElementwiseProduct + FeatureHasher + FValueSelector + FValueSelectorModel + HashingTF + IDF + IDFModel + Imputer + ImputerModel + IndexToString + Interaction + MaxAbsScaler + MaxAbsScalerModel + MinHashLSH + MinHashLSHModel + MinMaxScaler + MinMaxScalerModel + NGram + Normalizer + OneHotEncoder + OneHotEncoderModel + PCA + PCAModel + PolynomialExpansion + QuantileDiscretizer + RobustScaler + RobustScalerModel + RegexTokenizer + RFormula + RFormulaModel + SQLTransformer + StandardScaler + StandardScalerModel + StopWordsRemover + StringIndexer + StringIndexerModel + Tokenizer + VarianceThresholdSelector + VarianceThresholdSelectorModel + VectorAssembler + VectorIndexer + VectorIndexerModel + VectorSizeHint + VectorSlicer + Word2Vec + Word2VecModel + + +Classification +-------------- + +.. currentmodule:: pyspark.ml.classification + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + LinearSVC + LinearSVCModel + LinearSVCSummary + LinearSVCTrainingSummary + LogisticRegression + LogisticRegressionModel + LogisticRegressionSummary + LogisticRegressionTrainingSummary + BinaryLogisticRegressionSummary + BinaryLogisticRegressionTrainingSummary + DecisionTreeClassifier + DecisionTreeClassificationModel + GBTClassifier + GBTClassificationModel + RandomForestClassifier + RandomForestClassificationModel + RandomForestClassificationSummary + RandomForestClassificationTrainingSummary + BinaryRandomForestClassificationSummary + BinaryRandomForestClassificationTrainingSummary + NaiveBayes + NaiveBayesModel + MultilayerPerceptronClassifier + MultilayerPerceptronClassificationModel + MultilayerPerceptronClassificationSummary + MultilayerPerceptronClassificationTrainingSummary + OneVsRest + OneVsRestModel + FMClassifier + FMClassificationModel + FMClassificationSummary + FMClassificationTrainingSummary + + +Clustering +---------- + +.. currentmodule:: pyspark.ml.clustering + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BisectingKMeans + BisectingKMeansModel + BisectingKMeansSummary + KMeans + KMeansModel + GaussianMixture + GaussianMixtureModel + GaussianMixtureSummary + LDA + LDAModel + LocalLDAModel + DistributedLDAModel + PowerIterationClustering + + +ML Functions +---------------------------- + +.. currentmodule:: pyspark.ml.functions + +.. autosummary:: + :toctree: api/ + + vector_to_array + + +Vector and Matrix +----------------- + +.. currentmodule:: pyspark.ml.linalg + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Vector + DenseVector + SparseVector + Vectors + Matrix + DenseMatrix + SparseMatrix + Matrices + + +Recommendation +-------------- + +.. currentmodule:: pyspark.ml.recommendation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ALS + ALSModel + + +Regression +---------- + +.. currentmodule:: pyspark.ml.regression + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + AFTSurvivalRegression + AFTSurvivalRegressionModel + DecisionTreeRegressor + DecisionTreeRegressionModel + GBTRegressor + GBTRegressionModel + GeneralizedLinearRegression + GeneralizedLinearRegressionModel + GeneralizedLinearRegressionSummary + GeneralizedLinearRegressionTrainingSummary + IsotonicRegression + IsotonicRegressionModel + LinearRegression + LinearRegressionModel + LinearRegressionSummary + LinearRegressionTrainingSummary + RandomForestRegressor + RandomForestRegressionModel + FMRegressor + FMRegressionModel + + +Statistics +---------- + +.. currentmodule:: pyspark.ml.stat + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ANOVATest + ChiSquareTest + Correlation + FValueTest + KolmogorovSmirnovTest + MultivariateGaussian + Summarizer + SummaryBuilder + + +Tuning +------ + +.. currentmodule:: pyspark.ml.tuning + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ParamGridBuilder + CrossValidator + CrossValidatorModel + TrainValidationSplit + TrainValidationSplitModel + + +Evaluation +---------- + +.. currentmodule:: pyspark.ml.evaluation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Evaluator + BinaryClassificationEvaluator + RegressionEvaluator + MulticlassClassificationEvaluator + MultilabelClassificationEvaluator + ClusteringEvaluator + RankingEvaluator + + +Frequency Pattern Mining +---------------------------- + +.. currentmodule:: pyspark.ml.fpm + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + FPGrowth + FPGrowthModel + PrefixSpan + + +Image +----- + +.. currentmodule:: pyspark.ml.image + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ImageSchema + _ImageSchema + + +Utilities +--------- + +.. currentmodule:: pyspark.ml.util + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BaseReadWrite + DefaultParamsReadable + DefaultParamsReader + DefaultParamsWritable + DefaultParamsWriter + GeneralMLWriter + HasTrainingSummary + Identifiable + MLReadable + MLReader + MLWritable + MLWriter + diff --git a/python/docs/source/reference/pyspark.mllib.rst b/python/docs/source/reference/pyspark.mllib.rst new file mode 100644 index 0000000000000..1251b1df752c7 --- /dev/null +++ b/python/docs/source/reference/pyspark.mllib.rst @@ -0,0 +1,253 @@ +.. 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. + + +MLlib +===== + +Classification +-------------- + +.. currentmodule:: pyspark.mllib.classification + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + LogisticRegressionModel + LogisticRegressionWithSGD + LogisticRegressionWithLBFGS + SVMModel + SVMWithSGD + NaiveBayesModel + NaiveBayes + StreamingLogisticRegressionWithSGD + + +Clustering +---------- + +.. currentmodule:: pyspark.mllib.clustering + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + + BisectingKMeansModel + BisectingKMeans + KMeansModel + KMeans + GaussianMixtureModel + GaussianMixture + PowerIterationClusteringModel + PowerIterationClustering + StreamingKMeans + StreamingKMeansModel + LDA + LDAModel + + +Evaluation +---------- + +.. currentmodule:: pyspark.mllib.evaluation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BinaryClassificationMetrics + RegressionMetrics + MulticlassMetrics + RankingMetrics + + +Feature +------- + +.. currentmodule:: pyspark.mllib.feature + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Normalizer + StandardScalerModel + StandardScaler + HashingTF + IDFModel + IDF + Word2Vec + Word2VecModel + ChiSqSelector + ChiSqSelectorModel + ElementwiseProduct + + +Frequency Pattern Mining +------------------------ + +.. currentmodule:: pyspark.mllib.fpm + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + FPGrowth + FPGrowthModel + PrefixSpan + PrefixSpanModel + + +Vector and Matrix +----------------- + +.. currentmodule:: pyspark.mllib.linalg + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Vector + DenseVector + SparseVector + Vectors + Matrix + DenseMatrix + SparseMatrix + Matrices + QRDecomposition + + +Distributed Representation +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. currentmodule:: pyspark.mllib.linalg.distributed + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + BlockMatrix + CoordinateMatrix + DistributedMatrix + IndexedRow + IndexedRowMatrix + MatrixEntry + RowMatrix + SingularValueDecomposition + + +Random +------ + +.. currentmodule:: pyspark.mllib.random + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + RandomRDDs + + +Recommendation +-------------- + +.. currentmodule:: pyspark.mllib.recommendation + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + MatrixFactorizationModel + ALS + Rating + + +Regression +---------- + +.. currentmodule:: pyspark.mllib.regression + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + LabeledPoint + LinearModel + LinearRegressionModel + LinearRegressionWithSGD + RidgeRegressionModel + RidgeRegressionWithSGD + LassoModel + LassoWithSGD + IsotonicRegressionModel + IsotonicRegression + StreamingLinearAlgorithm + StreamingLinearRegressionWithSGD + + +Statistics +---------- + +.. currentmodule:: pyspark.mllib.stat + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + Statistics + MultivariateStatisticalSummary + ChiSqTestResult + MultivariateGaussian + KernelDensity + + +Tree +---- + +.. currentmodule:: pyspark.mllib.tree + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + DecisionTreeModel + DecisionTree + RandomForestModel + RandomForest + GradientBoostedTreesModel + GradientBoostedTrees + + +Utilities +--------- + +.. currentmodule:: pyspark.mllib.util + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + JavaLoader + JavaSaveable + LinearDataGenerator + Loader + MLUtils + Saveable + diff --git a/python/docs/source/reference/pyspark.resource.rst b/python/docs/source/reference/pyspark.resource.rst new file mode 100644 index 0000000000000..a1d885c44c480 --- /dev/null +++ b/python/docs/source/reference/pyspark.resource.rst @@ -0,0 +1,38 @@ +.. 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. + + +=================== +Resource Management +=================== + +Core Classes +------------ + +.. currentmodule:: pyspark.resource + +.. autosummary:: + :toctree: api/ + + ResourceInformation + ResourceProfile + ResourceProfileBuilder + ExecutorResourceRequest + ExecutorResourceRequests + TaskResourceRequest + TaskResourceRequests + diff --git a/python/docs/source/reference/pyspark.rst b/python/docs/source/reference/pyspark.rst new file mode 100644 index 0000000000000..b50ae37b99690 --- /dev/null +++ b/python/docs/source/reference/pyspark.rst @@ -0,0 +1,275 @@ +.. 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 Core +========== + +Public Classes +-------------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + SparkContext + RDD + Broadcast + Accumulator + SparkConf + SparkFiles + StorageLevel + TaskContext + RDDBarrier + BarrierTaskContext + BarrierTaskInfo + + +Spark Context APIs +------------------ + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + SparkContext.PACKAGE_EXTENSIONS + SparkContext.accumulator + SparkContext.addFile + SparkContext.addPyFile + SparkContext.applicationId + SparkContext.binaryFiles + SparkContext.binaryRecords + SparkContext.broadcast + SparkContext.cancelAllJobs + SparkContext.cancelJobGroup + SparkContext.defaultMinPartitions + SparkContext.defaultParallelism + SparkContext.dump_profiles + SparkContext.emptyRDD + SparkContext.getConf + SparkContext.getLocalProperty + SparkContext.getOrCreate + SparkContext.hadoopFile + SparkContext.hadoopRDD + SparkContext.newAPIHadoopFile + SparkContext.newAPIHadoopRDD + SparkContext.parallelize + SparkContext.pickleFile + SparkContext.range + SparkContext.resources + SparkContext.runJob + SparkContext.sequenceFile + SparkContext.setCheckpointDir + SparkContext.setJobDescription + SparkContext.setJobGroup + SparkContext.setLocalProperty + SparkContext.setLogLevel + SparkContext.setSystemProperty + SparkContext.show_profiles + SparkContext.sparkUser + SparkContext.startTime + SparkContext.statusTracker + SparkContext.stop + SparkContext.textFile + SparkContext.uiWebUrl + SparkContext.union + SparkContext.version + SparkContext.wholeTextFiles + + +RDD APIs +-------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + RDD.aggregate + RDD.aggregateByKey + RDD.barrier + RDD.cache + RDD.cartesian + RDD.checkpoint + RDD.coalesce + RDD.cogroup + RDD.collect + RDD.collectAsMap + RDD.collectWithJobGroup + RDD.combineByKey + RDD.context + RDD.count + RDD.countApprox + RDD.countApproxDistinct + RDD.countByKey + RDD.countByValue + RDD.distinct + RDD.filter + RDD.first + RDD.flatMap + RDD.flatMapValues + RDD.fold + RDD.foldByKey + RDD.foreach + RDD.foreachPartition + RDD.fullOuterJoin + RDD.getCheckpointFile + RDD.getNumPartitions + RDD.getResourceProfile + RDD.getStorageLevel + RDD.glom + RDD.groupBy + RDD.groupByKey + RDD.groupWith + RDD.histogram + RDD.id + RDD.intersection + RDD.isCheckpointed + RDD.isEmpty + RDD.isLocallyCheckpointed + RDD.join + RDD.keyBy + RDD.keys + RDD.leftOuterJoin + RDD.localCheckpoint + RDD.lookup + RDD.map + RDD.mapPartitions + RDD.mapPartitionsWithIndex + RDD.mapPartitionsWithSplit + RDD.mapValues + RDD.max + RDD.mean + RDD.meanApprox + RDD.min + RDD.name + RDD.partitionBy + RDD.persist + RDD.pipe + RDD.randomSplit + RDD.reduce + RDD.reduceByKey + RDD.reduceByKeyLocally + RDD.repartition + RDD.repartitionAndSortWithinPartitions + RDD.rightOuterJoin + RDD.sample + RDD.sampleByKey + RDD.sampleStdev + RDD.sampleVariance + RDD.saveAsHadoopDataset + RDD.saveAsHadoopFile + RDD.saveAsNewAPIHadoopDataset + RDD.saveAsNewAPIHadoopFile + RDD.saveAsPickleFile + RDD.saveAsSequenceFile + RDD.saveAsTextFile + RDD.setName + RDD.sortBy + RDD.sortByKey + RDD.stats + RDD.stdev + RDD.subtract + RDD.subtractByKey + RDD.sum + RDD.sumApprox + RDD.take + RDD.takeOrdered + RDD.takeSample + RDD.toDebugString + RDD.toLocalIterator + RDD.top + RDD.treeAggregate + RDD.treeReduce + RDD.union + RDD.unpersist + RDD.values + RDD.variance + RDD.withResources + RDD.zip + RDD.zipWithIndex + RDD.zipWithUniqueId + + +Broadcast and Accumulator +------------------------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + Broadcast.destroy + Broadcast.dump + Broadcast.load + Broadcast.load_from_path + Broadcast.unpersist + Broadcast.value + Accumulator.add + Accumulator.value + + +Management +---------- + +.. currentmodule:: pyspark + +.. autosummary:: + :toctree: api/ + + SparkConf.contains + SparkConf.get + SparkConf.getAll + SparkConf.set + SparkConf.setAll + SparkConf.setAppName + SparkConf.setExecutorEnv + SparkConf.setIfMissing + SparkConf.setMaster + SparkConf.setSparkHome + SparkConf.toDebugString + SparkFiles.get + SparkFiles.getRootDirectory + StorageLevel.DISK_ONLY + StorageLevel.DISK_ONLY_2 + StorageLevel.MEMORY_AND_DISK + StorageLevel.MEMORY_AND_DISK_2 + StorageLevel.MEMORY_ONLY + StorageLevel.MEMORY_ONLY_2 + StorageLevel.OFF_HEAP + TaskContext.attemptNumber + TaskContext.get + TaskContext.getLocalProperty + TaskContext.partitionId + TaskContext.resources + TaskContext.stageId + TaskContext.taskAttemptId + RDDBarrier.mapPartitions + RDDBarrier.mapPartitionsWithIndex + BarrierTaskContext.allGather + BarrierTaskContext.attemptNumber + BarrierTaskContext.barrier + BarrierTaskContext.get + BarrierTaskContext.getLocalProperty + BarrierTaskContext.getTaskInfos + BarrierTaskContext.partitionId + BarrierTaskContext.resources + BarrierTaskContext.stageId + BarrierTaskContext.taskAttemptId diff --git a/python/docs/source/reference/pyspark.sql.rst b/python/docs/source/reference/pyspark.sql.rst new file mode 100644 index 0000000000000..7e0357cf9d858 --- /dev/null +++ b/python/docs/source/reference/pyspark.sql.rst @@ -0,0 +1,542 @@ +.. 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 SQL +========= + +Core Classes +------------ + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + SparkSession + DataFrame + Column + Row + GroupedData + DataFrameNaFunctions + DataFrameStatFunctions + Window + + +Spark Session APIs +------------------ + +.. currentmodule:: pyspark.sql + +The entry point to programming Spark with the Dataset and DataFrame API. +To create a Spark session, you should use ``SparkSession.builder`` attribute. +See also :class:`SparkSession`. + +.. autosummary:: + :toctree: api/ + + SparkSession.builder.appName + SparkSession.builder.config + SparkSession.builder.enableHiveSupport + SparkSession.builder.getOrCreate + SparkSession.builder.master + SparkSession.catalog + SparkSession.conf + SparkSession.createDataFrame + SparkSession.getActiveSession + SparkSession.newSession + SparkSession.range + SparkSession.read + SparkSession.readStream + SparkSession.sparkContext + SparkSession.sql + SparkSession.stop + SparkSession.streams + SparkSession.table + SparkSession.udf + SparkSession.version + + +Input and Output +---------------- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + DataFrameReader.csv + DataFrameReader.format + DataFrameReader.jdbc + DataFrameReader.json + DataFrameReader.load + DataFrameReader.option + DataFrameReader.options + DataFrameReader.orc + DataFrameReader.parquet + DataFrameReader.schema + DataFrameReader.table + DataFrameWriter.bucketBy + DataFrameWriter.csv + DataFrameWriter.format + DataFrameWriter.insertInto + DataFrameWriter.jdbc + DataFrameWriter.json + DataFrameWriter.mode + DataFrameWriter.option + DataFrameWriter.options + DataFrameWriter.orc + DataFrameWriter.parquet + DataFrameWriter.partitionBy + DataFrameWriter.save + DataFrameWriter.saveAsTable + DataFrameWriter.sortBy + DataFrameWriter.text + + +DataFrame APIs +-------------- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + DataFrame.agg + DataFrame.alias + DataFrame.approxQuantile + DataFrame.cache + DataFrame.checkpoint + DataFrame.coalesce + DataFrame.colRegex + DataFrame.collect + DataFrame.columns + DataFrame.corr + DataFrame.count + DataFrame.cov + DataFrame.createGlobalTempView + DataFrame.createOrReplaceGlobalTempView + DataFrame.createOrReplaceTempView + DataFrame.createTempView + DataFrame.crossJoin + DataFrame.crosstab + DataFrame.cube + DataFrame.describe + DataFrame.distinct + DataFrame.drop + DataFrame.dropDuplicates + DataFrame.drop_duplicates + DataFrame.dropna + DataFrame.dtypes + DataFrame.exceptAll + DataFrame.explain + DataFrame.fillna + DataFrame.filter + DataFrame.first + DataFrame.foreach + DataFrame.foreachPartition + DataFrame.freqItems + DataFrame.groupBy + DataFrame.head + DataFrame.hint + DataFrame.inputFiles + DataFrame.intersect + DataFrame.intersectAll + DataFrame.isLocal + DataFrame.isStreaming + DataFrame.join + DataFrame.limit + DataFrame.localCheckpoint + DataFrame.mapInPandas + DataFrame.na + DataFrame.orderBy + DataFrame.persist + DataFrame.printSchema + DataFrame.randomSplit + DataFrame.rdd + DataFrame.registerTempTable + DataFrame.repartition + DataFrame.repartitionByRange + DataFrame.replace + DataFrame.rollup + DataFrame.sameSemantics + DataFrame.sample + DataFrame.sampleBy + DataFrame.schema + DataFrame.select + DataFrame.selectExpr + DataFrame.semanticHash + DataFrame.show + DataFrame.sort + DataFrame.sortWithinPartitions + DataFrame.stat + DataFrame.storageLevel + DataFrame.subtract + DataFrame.summary + DataFrame.tail + DataFrame.take + DataFrame.toDF + DataFrame.toJSON + DataFrame.toLocalIterator + DataFrame.toPandas + DataFrame.transform + DataFrame.union + DataFrame.unionAll + DataFrame.unionByName + DataFrame.unpersist + DataFrame.where + DataFrame.withColumn + DataFrame.withColumnRenamed + DataFrame.withWatermark + DataFrame.write + DataFrame.writeStream + DataFrame.writeTo + DataFrameNaFunctions.drop + DataFrameNaFunctions.fill + DataFrameNaFunctions.replace + DataFrameStatFunctions.approxQuantile + DataFrameStatFunctions.corr + DataFrameStatFunctions.cov + DataFrameStatFunctions.crosstab + DataFrameStatFunctions.freqItems + DataFrameStatFunctions.sampleBy + + +Data Types +---------- + +.. currentmodule:: pyspark.sql.types + +.. autosummary:: + :template: class_with_docs.rst + :toctree: api/ + + ArrayType + BinaryType + BooleanType + ByteType + DataType + DateType + DecimalType + DoubleType + FloatType + IntegerType + LongType + MapType + NullType + ShortType + StringType + StructField + StructType + TimestampType + + +Row +--- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + Row.asDict + + +Functions +--------- + +.. currentmodule:: pyspark.sql.functions + +.. autosummary:: + :toctree: api/ + + abs + acos + add_months + aggregate + approxCountDistinct + approx_count_distinct + array + array_contains + array_distinct + array_except + array_intersect + array_join + array_max + array_min + array_position + array_remove + array_repeat + array_sort + array_union + arrays_overlap + arrays_zip + asc + asc_nulls_first + asc_nulls_last + ascii + asin + atan + atan2 + avg + base64 + bin + bitwiseNOT + broadcast + bround + bucket + cbrt + ceil + coalesce + col + collect_list + collect_set + column + concat + concat_ws + conv + corr + cos + cosh + count + countDistinct + covar_pop + covar_samp + crc32 + create_map + cume_dist + current_date + current_timestamp + date_add + date_format + date_sub + date_trunc + datediff + dayofmonth + dayofweek + dayofyear + days + decode + degrees + dense_rank + desc + desc_nulls_first + desc_nulls_last + element_at + encode + exists + exp + explode + explode_outer + expm1 + expr + factorial + filter + first + flatten + floor + forall + format_number + format_string + from_csv + from_json + from_unixtime + from_utc_timestamp + get_json_object + greatest + grouping + grouping_id + hash + hex + hour + hours + hypot + initcap + input_file_name + instr + isnan + isnull + json_tuple + kurtosis + lag + last + last_day + lead + least + length + levenshtein + lit + locate + log + log10 + log1p + log2 + lower + lpad + ltrim + map_concat + map_entries + map_filter + map_from_arrays + map_from_entries + map_keys + map_values + map_zip_with + max + md5 + mean + min + minute + monotonically_increasing_id + month + months + months_between + nanvl + next_day + ntile + overlay + pandas_udf + percent_rank + percentile_approx + posexplode + posexplode_outer + pow + quarter + radians + rand + randn + rank + regexp_extract + regexp_replace + repeat + reverse + rint + round + row_number + rpad + rtrim + schema_of_csv + schema_of_json + second + sequence + sha1 + sha2 + shiftLeft + shiftRight + shiftRightUnsigned + shuffle + signum + sin + sinh + size + skewness + slice + sort_array + soundex + spark_partition_id + split + sqrt + stddev + stddev_pop + stddev_samp + struct + substring + substring_index + sum + sumDistinct + tan + tanh + timestamp_seconds + toDegrees + toRadians + to_csv + to_date + to_json + to_timestamp + to_utc_timestamp + transform + transform_keys + transform_values + translate + trim + trunc + udf + unbase64 + unhex + unix_timestamp + upper + var_pop + var_samp + variance + weekofyear + when + window + xxhash64 + year + years + zip_with + + +.. currentmodule:: pyspark.sql.avro.functions + +.. autosummary:: + :toctree: api/ + + from_avro + to_avro + +Window +------ + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + Window.currentRow + Window.orderBy + Window.partitionBy + Window.rangeBetween + Window.rowsBetween + Window.unboundedFollowing + Window.unboundedPreceding + WindowSpec.orderBy + WindowSpec.partitionBy + WindowSpec.rangeBetween + WindowSpec.rowsBetween + +Grouping +-------- + +.. currentmodule:: pyspark.sql + +.. autosummary:: + :toctree: api/ + + GroupedData.agg + GroupedData.apply + GroupedData.applyInPandas + GroupedData.avg + GroupedData.cogroup + GroupedData.count + GroupedData.max + GroupedData.mean + GroupedData.min + GroupedData.pivot + GroupedData.sum + diff --git a/python/docs/source/reference/pyspark.ss.rst b/python/docs/source/reference/pyspark.ss.rst new file mode 100644 index 0000000000000..a7936a4f2a59c --- /dev/null +++ b/python/docs/source/reference/pyspark.ss.rst @@ -0,0 +1,90 @@ +.. 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. + + +==================== +Structured Streaming +==================== + +Core Classes +------------ + +.. currentmodule:: pyspark.sql.streaming + +.. autosummary:: + :toctree: api/ + + DataStreamReader + DataStreamWriter + ForeachBatchFunction + StreamingQuery + StreamingQueryException + StreamingQueryManager + +Input and Output +---------------- + +.. currentmodule:: pyspark.sql.streaming + +.. autosummary:: + :toctree: api/ + + DataStreamReader.csv + DataStreamReader.format + DataStreamReader.json + DataStreamReader.load + DataStreamReader.option + DataStreamReader.options + DataStreamReader.orc + DataStreamReader.parquet + DataStreamReader.schema + DataStreamReader.text + DataStreamWriter.foreach + DataStreamWriter.foreachBatch + DataStreamWriter.format + DataStreamWriter.option + DataStreamWriter.options + DataStreamWriter.outputMode + DataStreamWriter.partitionBy + DataStreamWriter.queryName + DataStreamWriter.start + DataStreamWriter.trigger + +Query Management +---------------- + +.. currentmodule:: pyspark.sql.streaming + +.. autosummary:: + :toctree: api/ + + StreamingQuery.awaitTermination + StreamingQuery.exception + StreamingQuery.explain + StreamingQuery.id + StreamingQuery.isActive + StreamingQuery.lastProgress + StreamingQuery.name + StreamingQuery.processAllAvailable + StreamingQuery.recentProgress + StreamingQuery.runId + StreamingQuery.status + StreamingQuery.stop + StreamingQueryManager.active + StreamingQueryManager.awaitAnyTermination + StreamingQueryManager.get + StreamingQueryManager.resetTerminated diff --git a/python/docs/source/reference/pyspark.streaming.rst b/python/docs/source/reference/pyspark.streaming.rst new file mode 100644 index 0000000000000..57cbd00b67e4c --- /dev/null +++ b/python/docs/source/reference/pyspark.streaming.rst @@ -0,0 +1,130 @@ +.. 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 +=============== + +Core Classes +------------ + +.. currentmodule:: pyspark.streaming + +.. autosummary:: + :toctree: api/ + + StreamingContext + DStream + + +Streaming Management +-------------------- + +.. currentmodule:: pyspark.streaming + +.. autosummary:: + :toctree: api/ + + StreamingContext.addStreamingListener + StreamingContext.awaitTermination + StreamingContext.awaitTerminationOrTimeout + StreamingContext.checkpoint + StreamingContext.getActive + StreamingContext.getActiveOrCreate + StreamingContext.getOrCreate + StreamingContext.remember + StreamingContext.sparkContext + StreamingContext.start + StreamingContext.stop + StreamingContext.transform + StreamingContext.union + + +Input and Output +---------------- + +.. autosummary:: + :toctree: api/ + + StreamingContext.binaryRecordsStream + StreamingContext.queueStream + StreamingContext.socketTextStream + StreamingContext.textFileStream + DStream.pprint + DStream.saveAsTextFiles + + +Transformations and Actions +--------------------------- + +.. currentmodule:: pyspark.streaming + +.. autosummary:: + :toctree: api/ + + DStream.cache + DStream.checkpoint + DStream.cogroup + DStream.combineByKey + DStream.context + DStream.count + DStream.countByValue + DStream.countByValueAndWindow + DStream.countByWindow + DStream.filter + DStream.flatMap + DStream.flatMapValues + DStream.foreachRDD + DStream.fullOuterJoin + DStream.glom + DStream.groupByKey + DStream.groupByKeyAndWindow + DStream.join + DStream.leftOuterJoin + DStream.map + DStream.mapPartitions + DStream.mapPartitionsWithIndex + DStream.mapValues + DStream.partitionBy + DStream.persist + DStream.reduce + DStream.reduceByKey + DStream.reduceByKeyAndWindow + DStream.reduceByWindow + DStream.repartition + DStream.rightOuterJoin + DStream.slice + DStream.transform + DStream.transformWith + DStream.union + DStream.updateStateByKey + DStream.window + + +Kinesis +------- + +.. currentmodule:: pyspark.streaming.kinesis + +.. autosummary:: + :toctree: api/ + + KinesisUtils.createStream + InitialPositionInStream.LATEST + InitialPositionInStream.TRIM_HORIZON + diff --git a/python/docs/source/user_guide/index.rst b/python/docs/source/user_guide/index.rst new file mode 100644 index 0000000000000..e8a8d905f46d1 --- /dev/null +++ b/python/docs/source/user_guide/index.rst @@ -0,0 +1,22 @@ +.. 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. + + +========== +User Guide +========== + diff --git a/python/pylintrc b/python/pylintrc index 26d2741d3b56f..548377416ca3d 100644 --- a/python/pylintrc +++ b/python/pylintrc @@ -29,7 +29,7 @@ profile=no # Add files or directories to the ignoreList. They should be base names, not # paths. -ignore=pyspark.heapq3 +#ignore= # Pickle collected data for later comparisons. persistent=yes diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index ee153af18c88c..61e38fdb2a57b 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -42,6 +42,8 @@ A :class:`TaskContext` that provides extra info and tooling for barrier execution. - :class:`BarrierTaskInfo`: Information about a barrier task. + - :class:`InheritableThread`: + A inheritable thread to use in Spark when the pinned thread mode is on. """ from functools import wraps @@ -51,6 +53,7 @@ from pyspark.context import SparkContext from pyspark.rdd import RDD, RDDBarrier from pyspark.files import SparkFiles +from pyspark.util import InheritableThread from pyspark.storagelevel import StorageLevel from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast @@ -118,5 +121,5 @@ def wrapper(self, *args, **kwargs): "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", - "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", + "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "InheritableThread", ] diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py deleted file mode 100644 index af49c77a2d98c..0000000000000 --- a/python/pyspark/cloudpickle.py +++ /dev/null @@ -1,1362 +0,0 @@ -""" -This class is defined to override standard pickle functionality - -The goals of it follow: --Serialize lambdas and nested functions to compiled byte code --Deal with main module correctly --Deal with other non-serializable objects - -It does not include an unpickler, as standard python unpickling suffices. - -This module was extracted from the `cloud` package, developed by `PiCloud, Inc. -`_. - -Copyright (c) 2012, Regents of the University of California. -Copyright (c) 2009 `PiCloud, Inc. `_. -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - * Neither the name of the University of California, Berkeley nor the - names of its contributors may be used to endorse or promote - products derived from this software without specific prior written - permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED -TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -""" -from __future__ import print_function - -import dis -from functools import partial -import io -import itertools -import logging -import opcode -import operator -import pickle -import struct -import sys -import traceback -import types -import weakref -import uuid -import threading - - -try: - from enum import Enum -except ImportError: - Enum = None - -# cloudpickle is meant for inter process communication: we expect all -# communicating processes to run the same Python version hence we favor -# communication speed over compatibility: -DEFAULT_PROTOCOL = pickle.HIGHEST_PROTOCOL - -# Track the provenance of reconstructed dynamic classes to make it possible to -# recontruct instances from the matching singleton class definition when -# appropriate and preserve the usual "isinstance" semantics of Python objects. -_DYNAMIC_CLASS_TRACKER_BY_CLASS = weakref.WeakKeyDictionary() -_DYNAMIC_CLASS_TRACKER_BY_ID = weakref.WeakValueDictionary() -_DYNAMIC_CLASS_TRACKER_LOCK = threading.Lock() - -if sys.version_info[0] < 3: # pragma: no branch - from pickle import Pickler - try: - from cStringIO import StringIO - except ImportError: - from StringIO import StringIO - string_types = (basestring,) # noqa - PY3 = False - PY2 = True - PY2_WRAPPER_DESCRIPTOR_TYPE = type(object.__init__) - PY2_METHOD_WRAPPER_TYPE = type(object.__eq__) - PY2_CLASS_DICT_SKIP_PICKLE_METHOD_TYPE = (PY2_METHOD_WRAPPER_TYPE, - PY2_WRAPPER_DESCRIPTOR_TYPE) -else: - types.ClassType = type - from pickle import _Pickler as Pickler - from io import BytesIO as StringIO - string_types = (str,) - PY3 = True - PY2 = False - - -def _ensure_tracking(class_def): - with _DYNAMIC_CLASS_TRACKER_LOCK: - class_tracker_id = _DYNAMIC_CLASS_TRACKER_BY_CLASS.get(class_def) - if class_tracker_id is None: - class_tracker_id = uuid.uuid4().hex - _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id - _DYNAMIC_CLASS_TRACKER_BY_ID[class_tracker_id] = class_def - return class_tracker_id - - -def _lookup_class_or_track(class_tracker_id, class_def): - if class_tracker_id is not None: - with _DYNAMIC_CLASS_TRACKER_LOCK: - class_def = _DYNAMIC_CLASS_TRACKER_BY_ID.setdefault( - class_tracker_id, class_def) - _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id - return class_def - - -def _make_cell_set_template_code(): - """Get the Python compiler to emit LOAD_FAST(arg); STORE_DEREF - - Notes - ----- - In Python 3, we could use an easier function: - - .. code-block:: python - - def f(): - cell = None - - def _stub(value): - nonlocal cell - cell = value - - return _stub - - _cell_set_template_code = f().__code__ - - This function is _only_ a LOAD_FAST(arg); STORE_DEREF, but that is - invalid syntax on Python 2. If we use this function we also don't need - to do the weird freevars/cellvars swap below - """ - def inner(value): - lambda: cell # make ``cell`` a closure so that we get a STORE_DEREF - cell = value - - co = inner.__code__ - - # NOTE: we are marking the cell variable as a free variable intentionally - # so that we simulate an inner function instead of the outer function. This - # is what gives us the ``nonlocal`` behavior in a Python 2 compatible way. - if PY2: # pragma: no branch - return types.CodeType( - co.co_argcount, - co.co_nlocals, - co.co_stacksize, - co.co_flags, - co.co_code, - co.co_consts, - co.co_names, - co.co_varnames, - co.co_filename, - co.co_name, - co.co_firstlineno, - co.co_lnotab, - co.co_cellvars, # this is the trickery - (), - ) - else: - if hasattr(types.CodeType, "co_posonlyargcount"): # pragma: no branch - return types.CodeType( - co.co_argcount, - co.co_posonlyargcount, # Python3.8 with PEP570 - co.co_kwonlyargcount, - co.co_nlocals, - co.co_stacksize, - co.co_flags, - co.co_code, - co.co_consts, - co.co_names, - co.co_varnames, - co.co_filename, - co.co_name, - co.co_firstlineno, - co.co_lnotab, - co.co_cellvars, # this is the trickery - (), - ) - else: - return types.CodeType( - co.co_argcount, - co.co_kwonlyargcount, - co.co_nlocals, - co.co_stacksize, - co.co_flags, - co.co_code, - co.co_consts, - co.co_names, - co.co_varnames, - co.co_filename, - co.co_name, - co.co_firstlineno, - co.co_lnotab, - co.co_cellvars, # this is the trickery - (), - ) - -_cell_set_template_code = _make_cell_set_template_code() - - -def cell_set(cell, value): - """Set the value of a closure cell. - """ - return types.FunctionType( - _cell_set_template_code, - {}, - '_cell_set_inner', - (), - (cell,), - )(value) - - -# relevant opcodes -STORE_GLOBAL = opcode.opmap['STORE_GLOBAL'] -DELETE_GLOBAL = opcode.opmap['DELETE_GLOBAL'] -LOAD_GLOBAL = opcode.opmap['LOAD_GLOBAL'] -GLOBAL_OPS = (STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL) -HAVE_ARGUMENT = dis.HAVE_ARGUMENT -EXTENDED_ARG = dis.EXTENDED_ARG - - -def islambda(func): - return getattr(func, '__name__') == '' - - -_BUILTIN_TYPE_NAMES = {} -for k, v in types.__dict__.items(): - if type(v) is type: - _BUILTIN_TYPE_NAMES[v] = k - - -def _builtin_type(name): - return getattr(types, name) - - -def _make__new__factory(type_): - def _factory(): - return type_.__new__ - return _factory - - -# NOTE: These need to be module globals so that they're pickleable as globals. -_get_dict_new = _make__new__factory(dict) -_get_frozenset_new = _make__new__factory(frozenset) -_get_list_new = _make__new__factory(list) -_get_set_new = _make__new__factory(set) -_get_tuple_new = _make__new__factory(tuple) -_get_object_new = _make__new__factory(object) - -# Pre-defined set of builtin_function_or_method instances that can be -# serialized. -_BUILTIN_TYPE_CONSTRUCTORS = { - dict.__new__: _get_dict_new, - frozenset.__new__: _get_frozenset_new, - set.__new__: _get_set_new, - list.__new__: _get_list_new, - tuple.__new__: _get_tuple_new, - object.__new__: _get_object_new, -} - - -if sys.version_info < (3, 4): # pragma: no branch - def _walk_global_ops(code): - """ - Yield (opcode, argument number) tuples for all - global-referencing instructions in *code*. - """ - code = getattr(code, 'co_code', b'') - if PY2: # pragma: no branch - code = map(ord, code) - - n = len(code) - i = 0 - extended_arg = 0 - while i < n: - op = code[i] - i += 1 - if op >= HAVE_ARGUMENT: - oparg = code[i] + code[i + 1] * 256 + extended_arg - extended_arg = 0 - i += 2 - if op == EXTENDED_ARG: - extended_arg = oparg * 65536 - if op in GLOBAL_OPS: - yield op, oparg - -else: - def _walk_global_ops(code): - """ - Yield (opcode, argument number) tuples for all - global-referencing instructions in *code*. - """ - for instr in dis.get_instructions(code): - op = instr.opcode - if op in GLOBAL_OPS: - yield op, instr.arg - - -def _extract_class_dict(cls): - """Retrieve a copy of the dict of a class without the inherited methods""" - clsdict = dict(cls.__dict__) # copy dict proxy to a dict - if len(cls.__bases__) == 1: - inherited_dict = cls.__bases__[0].__dict__ - else: - inherited_dict = {} - for base in reversed(cls.__bases__): - inherited_dict.update(base.__dict__) - to_remove = [] - for name, value in clsdict.items(): - try: - base_value = inherited_dict[name] - if value is base_value: - to_remove.append(name) - elif PY2: - # backward compat for Python 2 - if hasattr(value, "im_func"): - if value.im_func is getattr(base_value, "im_func", None): - to_remove.append(name) - elif isinstance(value, PY2_CLASS_DICT_SKIP_PICKLE_METHOD_TYPE): - # On Python 2 we have no way to pickle those specific - # methods types nor to check that they are actually - # inherited. So we assume that they are always inherited - # from builtin types. - to_remove.append(name) - except KeyError: - pass - for name in to_remove: - clsdict.pop(name) - return clsdict - - -class CloudPickler(Pickler): - - dispatch = Pickler.dispatch.copy() - - def __init__(self, file, protocol=None): - if protocol is None: - protocol = DEFAULT_PROTOCOL - Pickler.__init__(self, file, protocol=protocol) - # map ids to dictionary. used to ensure that functions can share global env - self.globals_ref = {} - - def dump(self, obj): - self.inject_addons() - try: - return Pickler.dump(self, obj) - except RuntimeError as e: - if 'recursion' in e.args[0]: - msg = """Could not pickle object as excessively deep recursion required.""" - raise pickle.PicklingError(msg) - else: - raise - - def save_memoryview(self, obj): - self.save(obj.tobytes()) - - dispatch[memoryview] = save_memoryview - - if PY2: # pragma: no branch - def save_buffer(self, obj): - self.save(str(obj)) - - dispatch[buffer] = save_buffer # noqa: F821 'buffer' was removed in Python 3 - - def save_module(self, obj): - """ - Save a module as an import - """ - if _is_dynamic(obj): - self.save_reduce(dynamic_subimport, (obj.__name__, vars(obj)), - obj=obj) - else: - self.save_reduce(subimport, (obj.__name__,), obj=obj) - - dispatch[types.ModuleType] = save_module - - def save_codeobject(self, obj): - """ - Save a code object - """ - if PY3: # pragma: no branch - if hasattr(obj, "co_posonlyargcount"): # pragma: no branch - args = ( - obj.co_argcount, obj.co_posonlyargcount, - obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, - obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, - obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, - obj.co_cellvars - ) - else: - args = ( - obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, - obj.co_stacksize, obj.co_flags, obj.co_code, obj.co_consts, - obj.co_names, obj.co_varnames, obj.co_filename, - obj.co_name, obj.co_firstlineno, obj.co_lnotab, - obj.co_freevars, obj.co_cellvars - ) - else: - args = ( - obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, - obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars - ) - self.save_reduce(types.CodeType, args, obj=obj) - - dispatch[types.CodeType] = save_codeobject - - def save_function(self, obj, name=None): - """ Registered with the dispatch to handle all function types. - - Determines what kind of function obj is (e.g. lambda, defined at - interactive prompt, etc) and handles the pickling appropriately. - """ - try: - should_special_case = obj in _BUILTIN_TYPE_CONSTRUCTORS - except TypeError: - # Methods of builtin types aren't hashable in python 2. - should_special_case = False - - if should_special_case: - # We keep a special-cased cache of built-in type constructors at - # global scope, because these functions are structured very - # differently in different python versions and implementations (for - # example, they're instances of types.BuiltinFunctionType in - # CPython, but they're ordinary types.FunctionType instances in - # PyPy). - # - # If the function we've received is in that cache, we just - # serialize it as a lookup into the cache. - return self.save_reduce(_BUILTIN_TYPE_CONSTRUCTORS[obj], (), obj=obj) - - write = self.write - - if name is None: - name = obj.__name__ - try: - # whichmodule() could fail, see - # https://bitbucket.org/gutworth/six/issues/63/importing-six-breaks-pickling - modname = pickle.whichmodule(obj, name) - except Exception: - modname = None - # print('which gives %s %s %s' % (modname, obj, name)) - try: - themodule = sys.modules[modname] - except KeyError: - # eval'd items such as namedtuple give invalid items for their function __module__ - modname = '__main__' - - if modname == '__main__': - themodule = None - - try: - lookedup_by_name = getattr(themodule, name, None) - except Exception: - lookedup_by_name = None - - if themodule: - if lookedup_by_name is obj: - return self.save_global(obj, name) - - # a builtin_function_or_method which comes in as an attribute of some - # object (e.g., itertools.chain.from_iterable) will end - # up with modname "__main__" and so end up here. But these functions - # have no __code__ attribute in CPython, so the handling for - # user-defined functions below will fail. - # So we pickle them here using save_reduce; have to do it differently - # for different python versions. - if not hasattr(obj, '__code__'): - if PY3: # pragma: no branch - rv = obj.__reduce_ex__(self.proto) - else: - if hasattr(obj, '__self__'): - rv = (getattr, (obj.__self__, name)) - else: - raise pickle.PicklingError("Can't pickle %r" % obj) - return self.save_reduce(obj=obj, *rv) - - # if func is lambda, def'ed at prompt, is in main, or is nested, then - # we'll pickle the actual function object rather than simply saving a - # reference (as is done in default pickler), via save_function_tuple. - if (islambda(obj) - or getattr(obj.__code__, 'co_filename', None) == '' - or themodule is None): - self.save_function_tuple(obj) - return - else: - # func is nested - if lookedup_by_name is None or lookedup_by_name is not obj: - self.save_function_tuple(obj) - return - - if obj.__dict__: - # essentially save_reduce, but workaround needed to avoid recursion - self.save(_restore_attr) - write(pickle.MARK + pickle.GLOBAL + modname + '\n' + name + '\n') - self.memoize(obj) - self.save(obj.__dict__) - write(pickle.TUPLE + pickle.REDUCE) - else: - write(pickle.GLOBAL + modname + '\n' + name + '\n') - self.memoize(obj) - - dispatch[types.FunctionType] = save_function - - def _save_subimports(self, code, top_level_dependencies): - """ - Save submodules used by a function but not listed in its globals. - - In the example below: - - ``` - import concurrent.futures - import cloudpickle - - - def func(): - x = concurrent.futures.ThreadPoolExecutor - - - if __name__ == '__main__': - cloudpickle.dumps(func) - ``` - - the globals extracted by cloudpickle in the function's state include - the concurrent module, but not its submodule (here, - concurrent.futures), which is the module used by func. - - To ensure that calling the depickled function does not raise an - AttributeError, this function looks for any currently loaded submodule - that the function uses and whose parent is present in the function - globals, and saves it before saving the function. - """ - - # check if any known dependency is an imported package - for x in top_level_dependencies: - if isinstance(x, types.ModuleType) and hasattr(x, '__package__') and x.__package__: - # check if the package has any currently loaded sub-imports - prefix = x.__name__ + '.' - # A concurrent thread could mutate sys.modules, - # make sure we iterate over a copy to avoid exceptions - for name in list(sys.modules): - # Older versions of pytest will add a "None" module to sys.modules. - if name is not None and name.startswith(prefix): - # check whether the function can address the sub-module - tokens = set(name[len(prefix):].split('.')) - if not tokens - set(code.co_names): - # ensure unpickler executes this import - self.save(sys.modules[name]) - # then discards the reference to it - self.write(pickle.POP) - - def _save_dynamic_enum(self, obj, clsdict): - """Special handling for dynamic Enum subclasses - - Use a dedicated Enum constructor (inspired by EnumMeta.__call__) as the - EnumMeta metaclass has complex initialization that makes the Enum - subclasses hold references to their own instances. - """ - members = dict((e.name, e.value) for e in obj) - - # Python 2.7 with enum34 can have no qualname: - qualname = getattr(obj, "__qualname__", None) - - self.save_reduce(_make_skeleton_enum, - (obj.__bases__, obj.__name__, qualname, members, - obj.__module__, _ensure_tracking(obj), None), - obj=obj) - - # Cleanup the clsdict that will be passed to _rehydrate_skeleton_class: - # Those attributes are already handled by the metaclass. - for attrname in ["_generate_next_value_", "_member_names_", - "_member_map_", "_member_type_", - "_value2member_map_"]: - clsdict.pop(attrname, None) - for member in members: - clsdict.pop(member) - - def save_dynamic_class(self, obj): - """Save a class that can't be stored as module global. - - This method is used to serialize classes that are defined inside - functions, or that otherwise can't be serialized as attribute lookups - from global modules. - """ - clsdict = _extract_class_dict(obj) - clsdict.pop('__weakref__', None) - - # For ABCMeta in python3.7+, remove _abc_impl as it is not picklable. - # This is a fix which breaks the cache but this only makes the first - # calls to issubclass slower. - if "_abc_impl" in clsdict: - import abc - (registry, _, _, _) = abc._get_dump(obj) - clsdict["_abc_impl"] = [subclass_weakref() - for subclass_weakref in registry] - - # On PyPy, __doc__ is a readonly attribute, so we need to include it in - # the initial skeleton class. This is safe because we know that the - # doc can't participate in a cycle with the original class. - type_kwargs = {'__doc__': clsdict.pop('__doc__', None)} - - if hasattr(obj, "__slots__"): - type_kwargs['__slots__'] = obj.__slots__ - # pickle string length optimization: member descriptors of obj are - # created automatically from obj's __slots__ attribute, no need to - # save them in obj's state - if isinstance(obj.__slots__, string_types): - clsdict.pop(obj.__slots__) - else: - for k in obj.__slots__: - clsdict.pop(k, None) - - # If type overrides __dict__ as a property, include it in the type - # kwargs. In Python 2, we can't set this attribute after construction. - __dict__ = clsdict.pop('__dict__', None) - if isinstance(__dict__, property): - type_kwargs['__dict__'] = __dict__ - - save = self.save - write = self.write - - # We write pickle instructions explicitly here to handle the - # possibility that the type object participates in a cycle with its own - # __dict__. We first write an empty "skeleton" version of the class and - # memoize it before writing the class' __dict__ itself. We then write - # instructions to "rehydrate" the skeleton class by restoring the - # attributes from the __dict__. - # - # A type can appear in a cycle with its __dict__ if an instance of the - # type appears in the type's __dict__ (which happens for the stdlib - # Enum class), or if the type defines methods that close over the name - # of the type, (which is common for Python 2-style super() calls). - - # Push the rehydration function. - save(_rehydrate_skeleton_class) - - # Mark the start of the args tuple for the rehydration function. - write(pickle.MARK) - - # Create and memoize an skeleton class with obj's name and bases. - if Enum is not None and issubclass(obj, Enum): - # Special handling of Enum subclasses - self._save_dynamic_enum(obj, clsdict) - else: - # "Regular" class definition: - tp = type(obj) - self.save_reduce(_make_skeleton_class, - (tp, obj.__name__, obj.__bases__, type_kwargs, - _ensure_tracking(obj), None), - obj=obj) - - # Now save the rest of obj's __dict__. Any references to obj - # encountered while saving will point to the skeleton class. - save(clsdict) - - # Write a tuple of (skeleton_class, clsdict). - write(pickle.TUPLE) - - # Call _rehydrate_skeleton_class(skeleton_class, clsdict) - write(pickle.REDUCE) - - def save_function_tuple(self, func): - """ Pickles an actual func object. - - A func comprises: code, globals, defaults, closure, and dict. We - extract and save these, injecting reducing functions at certain points - to recreate the func object. Keep in mind that some of these pieces - can contain a ref to the func itself. Thus, a naive save on these - pieces could trigger an infinite loop of save's. To get around that, - we first create a skeleton func object using just the code (this is - safe, since this won't contain a ref to the func), and memoize it as - soon as it's created. The other stuff can then be filled in later. - """ - if is_tornado_coroutine(func): - self.save_reduce(_rebuild_tornado_coroutine, (func.__wrapped__,), - obj=func) - return - - save = self.save - write = self.write - - code, f_globals, defaults, closure_values, dct, base_globals = self.extract_func_data(func) - - save(_fill_function) # skeleton function updater - write(pickle.MARK) # beginning of tuple that _fill_function expects - - self._save_subimports( - code, - itertools.chain(f_globals.values(), closure_values or ()), - ) - - # create a skeleton function object and memoize it - save(_make_skel_func) - save(( - code, - len(closure_values) if closure_values is not None else -1, - base_globals, - )) - write(pickle.REDUCE) - self.memoize(func) - - # save the rest of the func data needed by _fill_function - state = { - 'globals': f_globals, - 'defaults': defaults, - 'dict': dct, - 'closure_values': closure_values, - 'module': func.__module__, - 'name': func.__name__, - 'doc': func.__doc__, - } - if hasattr(func, '__annotations__') and sys.version_info >= (3, 7): - state['annotations'] = func.__annotations__ - if hasattr(func, '__qualname__'): - state['qualname'] = func.__qualname__ - if hasattr(func, '__kwdefaults__'): - state['kwdefaults'] = func.__kwdefaults__ - save(state) - write(pickle.TUPLE) - write(pickle.REDUCE) # applies _fill_function on the tuple - - _extract_code_globals_cache = ( - weakref.WeakKeyDictionary() - if not hasattr(sys, "pypy_version_info") - else {}) - - @classmethod - def extract_code_globals(cls, co): - """ - Find all globals names read or written to by codeblock co - """ - out_names = cls._extract_code_globals_cache.get(co) - if out_names is None: - try: - names = co.co_names - except AttributeError: - # PyPy "builtin-code" object - out_names = set() - else: - out_names = {names[oparg] for _, oparg in _walk_global_ops(co)} - - # see if nested function have any global refs - if co.co_consts: - for const in co.co_consts: - if type(const) is types.CodeType: - out_names |= cls.extract_code_globals(const) - - cls._extract_code_globals_cache[co] = out_names - - return out_names - - def extract_func_data(self, func): - """ - Turn the function into a tuple of data necessary to recreate it: - code, globals, defaults, closure_values, dict - """ - code = func.__code__ - - # extract all global ref's - func_global_refs = self.extract_code_globals(code) - - # process all variables referenced by global environment - f_globals = {} - for var in func_global_refs: - if var in func.__globals__: - f_globals[var] = func.__globals__[var] - - # defaults requires no processing - defaults = func.__defaults__ - - # process closure - closure = ( - list(map(_get_cell_contents, func.__closure__)) - if func.__closure__ is not None - else None - ) - - # save the dict - dct = func.__dict__ - - # base_globals represents the future global namespace of func at - # unpickling time. Looking it up and storing it in globals_ref allow - # functions sharing the same globals at pickling time to also - # share them once unpickled, at one condition: since globals_ref is - # an attribute of a Cloudpickler instance, and that a new CloudPickler is - # created each time pickle.dump or pickle.dumps is called, functions - # also need to be saved within the same invokation of - # cloudpickle.dump/cloudpickle.dumps (for example: cloudpickle.dumps([f1, f2])). There - # is no such limitation when using Cloudpickler.dump, as long as the - # multiple invokations are bound to the same Cloudpickler. - base_globals = self.globals_ref.setdefault(id(func.__globals__), {}) - - if base_globals == {}: - # Add module attributes used to resolve relative imports - # instructions inside func. - for k in ["__package__", "__name__", "__path__", "__file__"]: - # Some built-in functions/methods such as object.__new__ have - # their __globals__ set to None in PyPy - if func.__globals__ is not None and k in func.__globals__: - base_globals[k] = func.__globals__[k] - - return (code, f_globals, defaults, closure, dct, base_globals) - - def save_builtin_function(self, obj): - if obj.__module__ == "__builtin__": - return self.save_global(obj) - return self.save_function(obj) - - dispatch[types.BuiltinFunctionType] = save_builtin_function - - def save_global(self, obj, name=None, pack=struct.pack): - """ - Save a "global". - - The name of this method is somewhat misleading: all types get - dispatched here. - """ - if obj is type(None): - return self.save_reduce(type, (None,), obj=obj) - elif obj is type(Ellipsis): - return self.save_reduce(type, (Ellipsis,), obj=obj) - elif obj is type(NotImplemented): - return self.save_reduce(type, (NotImplemented,), obj=obj) - - if obj.__module__ == "__main__": - return self.save_dynamic_class(obj) - - try: - return Pickler.save_global(self, obj, name=name) - except Exception: - if obj.__module__ == "__builtin__" or obj.__module__ == "builtins": - if obj in _BUILTIN_TYPE_NAMES: - return self.save_reduce( - _builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) - - typ = type(obj) - if typ is not obj and isinstance(obj, (type, types.ClassType)): - return self.save_dynamic_class(obj) - - raise - - dispatch[type] = save_global - dispatch[types.ClassType] = save_global - - def save_instancemethod(self, obj): - # Memoization rarely is ever useful due to python bounding - if obj.__self__ is None: - self.save_reduce(getattr, (obj.im_class, obj.__name__)) - else: - if PY3: # pragma: no branch - self.save_reduce(types.MethodType, (obj.__func__, obj.__self__), obj=obj) - else: - self.save_reduce(types.MethodType, (obj.__func__, obj.__self__, obj.__self__.__class__), - obj=obj) - - dispatch[types.MethodType] = save_instancemethod - - def save_inst(self, obj): - """Inner logic to save instance. Based off pickle.save_inst""" - cls = obj.__class__ - - # Try the dispatch table (pickle module doesn't do it) - f = self.dispatch.get(cls) - if f: - f(self, obj) # Call unbound method with explicit self - return - - memo = self.memo - write = self.write - save = self.save - - if hasattr(obj, '__getinitargs__'): - args = obj.__getinitargs__() - len(args) # XXX Assert it's a sequence - pickle._keep_alive(args, memo) - else: - args = () - - write(pickle.MARK) - - if self.bin: - save(cls) - for arg in args: - save(arg) - write(pickle.OBJ) - else: - for arg in args: - save(arg) - write(pickle.INST + cls.__module__ + '\n' + cls.__name__ + '\n') - - self.memoize(obj) - - try: - getstate = obj.__getstate__ - except AttributeError: - stuff = obj.__dict__ - else: - stuff = getstate() - pickle._keep_alive(stuff, memo) - save(stuff) - write(pickle.BUILD) - - if PY2: # pragma: no branch - dispatch[types.InstanceType] = save_inst - - def save_property(self, obj): - # properties not correctly saved in python - self.save_reduce(property, (obj.fget, obj.fset, obj.fdel, obj.__doc__), obj=obj) - - dispatch[property] = save_property - - def save_classmethod(self, obj): - orig_func = obj.__func__ - self.save_reduce(type(obj), (orig_func,), obj=obj) - - dispatch[classmethod] = save_classmethod - dispatch[staticmethod] = save_classmethod - - def save_itemgetter(self, obj): - """itemgetter serializer (needed for namedtuple support)""" - class Dummy: - def __getitem__(self, item): - return item - items = obj(Dummy()) - if not isinstance(items, tuple): - items = (items,) - return self.save_reduce(operator.itemgetter, items) - - if type(operator.itemgetter) is type: - dispatch[operator.itemgetter] = save_itemgetter - - def save_attrgetter(self, obj): - """attrgetter serializer""" - class Dummy(object): - def __init__(self, attrs, index=None): - self.attrs = attrs - self.index = index - def __getattribute__(self, item): - attrs = object.__getattribute__(self, "attrs") - index = object.__getattribute__(self, "index") - if index is None: - index = len(attrs) - attrs.append(item) - else: - attrs[index] = ".".join([attrs[index], item]) - return type(self)(attrs, index) - attrs = [] - obj(Dummy(attrs)) - return self.save_reduce(operator.attrgetter, tuple(attrs)) - - if type(operator.attrgetter) is type: - dispatch[operator.attrgetter] = save_attrgetter - - def save_file(self, obj): - """Save a file""" - try: - import StringIO as pystringIO # we can't use cStringIO as it lacks the name attribute - except ImportError: - import io as pystringIO - - if not hasattr(obj, 'name') or not hasattr(obj, 'mode'): - raise pickle.PicklingError("Cannot pickle files that do not map to an actual file") - if obj is sys.stdout: - return self.save_reduce(getattr, (sys, 'stdout'), obj=obj) - if obj is sys.stderr: - return self.save_reduce(getattr, (sys, 'stderr'), obj=obj) - if obj is sys.stdin: - raise pickle.PicklingError("Cannot pickle standard input") - if obj.closed: - raise pickle.PicklingError("Cannot pickle closed files") - if hasattr(obj, 'isatty') and obj.isatty(): - raise pickle.PicklingError("Cannot pickle files that map to tty objects") - if 'r' not in obj.mode and '+' not in obj.mode: - raise pickle.PicklingError("Cannot pickle files that are not opened for reading: %s" % obj.mode) - - name = obj.name - - retval = pystringIO.StringIO() - - try: - # Read the whole file - curloc = obj.tell() - obj.seek(0) - contents = obj.read() - obj.seek(curloc) - except IOError: - raise pickle.PicklingError("Cannot pickle file %s as it cannot be read" % name) - retval.write(contents) - retval.seek(curloc) - - retval.name = name - self.save(retval) - self.memoize(obj) - - def save_ellipsis(self, obj): - self.save_reduce(_gen_ellipsis, ()) - - def save_not_implemented(self, obj): - self.save_reduce(_gen_not_implemented, ()) - - try: # Python 2 - dispatch[file] = save_file - except NameError: # Python 3 # pragma: no branch - dispatch[io.TextIOWrapper] = save_file - - dispatch[type(Ellipsis)] = save_ellipsis - dispatch[type(NotImplemented)] = save_not_implemented - - def save_weakset(self, obj): - self.save_reduce(weakref.WeakSet, (list(obj),)) - - dispatch[weakref.WeakSet] = save_weakset - - def save_logger(self, obj): - self.save_reduce(logging.getLogger, (obj.name,), obj=obj) - - dispatch[logging.Logger] = save_logger - - def save_root_logger(self, obj): - self.save_reduce(logging.getLogger, (), obj=obj) - - dispatch[logging.RootLogger] = save_root_logger - - if hasattr(types, "MappingProxyType"): # pragma: no branch - def save_mappingproxy(self, obj): - self.save_reduce(types.MappingProxyType, (dict(obj),), obj=obj) - - dispatch[types.MappingProxyType] = save_mappingproxy - - """Special functions for Add-on libraries""" - def inject_addons(self): - """Plug in system. Register additional pickling functions if modules already loaded""" - pass - - -# Tornado support - -def is_tornado_coroutine(func): - """ - Return whether *func* is a Tornado coroutine function. - Running coroutines are not supported. - """ - if 'tornado.gen' not in sys.modules: - return False - gen = sys.modules['tornado.gen'] - if not hasattr(gen, "is_coroutine_function"): - # Tornado version is too old - return False - return gen.is_coroutine_function(func) - - -def _rebuild_tornado_coroutine(func): - from tornado import gen - return gen.coroutine(func) - - -# Shorthands for legacy support - -def dump(obj, file, protocol=None): - """Serialize obj as bytes streamed into file - - protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to - pickle.HIGHEST_PROTOCOL. This setting favors maximum communication speed - between processes running the same Python version. - - Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure - compatibility with older versions of Python. - """ - CloudPickler(file, protocol=protocol).dump(obj) - - -def dumps(obj, protocol=None): - """Serialize obj as a string of bytes allocated in memory - - protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to - pickle.HIGHEST_PROTOCOL. This setting favors maximum communication speed - between processes running the same Python version. - - Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure - compatibility with older versions of Python. - """ - file = StringIO() - try: - cp = CloudPickler(file, protocol=protocol) - cp.dump(obj) - return file.getvalue() - finally: - file.close() - - -# including pickles unloading functions in this namespace -load = pickle.load -loads = pickle.loads - - -# hack for __import__ not working as desired -def subimport(name): - __import__(name) - return sys.modules[name] - - -def dynamic_subimport(name, vars): - mod = types.ModuleType(name) - mod.__dict__.update(vars) - return mod - - -# restores function attributes -def _restore_attr(obj, attr): - for key, val in attr.items(): - setattr(obj, key, val) - return obj - - -def _gen_ellipsis(): - return Ellipsis - - -def _gen_not_implemented(): - return NotImplemented - - -def _get_cell_contents(cell): - try: - return cell.cell_contents - except ValueError: - # sentinel used by ``_fill_function`` which will leave the cell empty - return _empty_cell_value - - -def instance(cls): - """Create a new instance of a class. - - Parameters - ---------- - cls : type - The class to create an instance of. - - Returns - ------- - instance : cls - A new instance of ``cls``. - """ - return cls() - - -@instance -class _empty_cell_value(object): - """sentinel for empty closures - """ - @classmethod - def __reduce__(cls): - return cls.__name__ - - -def _fill_function(*args): - """Fills in the rest of function data into the skeleton function object - - The skeleton itself is create by _make_skel_func(). - """ - if len(args) == 2: - func = args[0] - state = args[1] - elif len(args) == 5: - # Backwards compat for cloudpickle v0.4.0, after which the `module` - # argument was introduced - func = args[0] - keys = ['globals', 'defaults', 'dict', 'closure_values'] - state = dict(zip(keys, args[1:])) - elif len(args) == 6: - # Backwards compat for cloudpickle v0.4.1, after which the function - # state was passed as a dict to the _fill_function it-self. - func = args[0] - keys = ['globals', 'defaults', 'dict', 'module', 'closure_values'] - state = dict(zip(keys, args[1:])) - else: - raise ValueError('Unexpected _fill_value arguments: %r' % (args,)) - - # - At pickling time, any dynamic global variable used by func is - # serialized by value (in state['globals']). - # - At unpickling time, func's __globals__ attribute is initialized by - # first retrieving an empty isolated namespace that will be shared - # with other functions pickled from the same original module - # by the same CloudPickler instance and then updated with the - # content of state['globals'] to populate the shared isolated - # namespace with all the global variables that are specifically - # referenced for this function. - func.__globals__.update(state['globals']) - - func.__defaults__ = state['defaults'] - func.__dict__ = state['dict'] - if 'annotations' in state: - func.__annotations__ = state['annotations'] - if 'doc' in state: - func.__doc__ = state['doc'] - if 'name' in state: - func.__name__ = state['name'] - if 'module' in state: - func.__module__ = state['module'] - if 'qualname' in state: - func.__qualname__ = state['qualname'] - if 'kwdefaults' in state: - func.__kwdefaults__ = state['kwdefaults'] - - cells = func.__closure__ - if cells is not None: - for cell, value in zip(cells, state['closure_values']): - if value is not _empty_cell_value: - cell_set(cell, value) - - return func - - -def _make_empty_cell(): - if False: - # trick the compiler into creating an empty cell in our lambda - cell = None - raise AssertionError('this route should not be executed') - - return (lambda: cell).__closure__[0] - - -def _make_skel_func(code, cell_count, base_globals=None): - """ Creates a skeleton function object that contains just the provided - code and the correct number of cells in func_closure. All other - func attributes (e.g. func_globals) are empty. - """ - # This is backward-compatibility code: for cloudpickle versions between - # 0.5.4 and 0.7, base_globals could be a string or None. base_globals - # should now always be a dictionary. - if base_globals is None or isinstance(base_globals, str): - base_globals = {} - - base_globals['__builtins__'] = __builtins__ - - closure = ( - tuple(_make_empty_cell() for _ in range(cell_count)) - if cell_count >= 0 else - None - ) - return types.FunctionType(code, base_globals, None, None, closure) - - -def _make_skeleton_class(type_constructor, name, bases, type_kwargs, - class_tracker_id, extra): - """Build dynamic class with an empty __dict__ to be filled once memoized - - If class_tracker_id is not None, try to lookup an existing class definition - matching that id. If none is found, track a newly reconstructed class - definition under that id so that other instances stemming from the same - class id will also reuse this class definition. - - The "extra" variable is meant to be a dict (or None) that can be used for - forward compatibility shall the need arise. - """ - skeleton_class = type_constructor(name, bases, type_kwargs) - return _lookup_class_or_track(class_tracker_id, skeleton_class) - - -def _rehydrate_skeleton_class(skeleton_class, class_dict): - """Put attributes from `class_dict` back on `skeleton_class`. - - See CloudPickler.save_dynamic_class for more info. - """ - registry = None - for attrname, attr in class_dict.items(): - if attrname == "_abc_impl": - registry = attr - else: - setattr(skeleton_class, attrname, attr) - if registry is not None: - for subclass in registry: - skeleton_class.register(subclass) - - return skeleton_class - - -def _make_skeleton_enum(bases, name, qualname, members, module, - class_tracker_id, extra): - """Build dynamic enum with an empty __dict__ to be filled once memoized - - The creation of the enum class is inspired by the code of - EnumMeta._create_. - - If class_tracker_id is not None, try to lookup an existing enum definition - matching that id. If none is found, track a newly reconstructed enum - definition under that id so that other instances stemming from the same - class id will also reuse this enum definition. - - The "extra" variable is meant to be a dict (or None) that can be used for - forward compatibility shall the need arise. - """ - # enums always inherit from their base Enum class at the last position in - # the list of base classes: - enum_base = bases[-1] - metacls = enum_base.__class__ - classdict = metacls.__prepare__(name, bases) - - for member_name, member_value in members.items(): - classdict[member_name] = member_value - enum_class = metacls.__new__(metacls, name, bases, classdict) - enum_class.__module__ = module - - # Python 2.7 compat - if qualname is not None: - enum_class.__qualname__ = qualname - - return _lookup_class_or_track(class_tracker_id, enum_class) - - -def _is_dynamic(module): - """ - Return True if the module is special module that cannot be imported by its - name. - """ - # Quick check: module that have __file__ attribute are not dynamic modules. - if hasattr(module, '__file__'): - return False - - if hasattr(module, '__spec__'): - return module.__spec__ is None - else: - # Backward compat for Python 2 - import imp - try: - path = None - for part in module.__name__.split('.'): - if path is not None: - path = [path] - f, path, description = imp.find_module(part, path) - if f is not None: - f.close() - except ImportError: - return True - return False - - -""" Use copy_reg to extend global pickle definitions """ - -if sys.version_info < (3, 4): # pragma: no branch - method_descriptor = type(str.upper) - - def _reduce_method_descriptor(obj): - return (getattr, (obj.__objclass__, obj.__name__)) - - try: - import copy_reg as copyreg - except ImportError: - import copyreg - copyreg.pickle(method_descriptor, _reduce_method_descriptor) \ No newline at end of file diff --git a/python/pyspark/cloudpickle/__init__.py b/python/pyspark/cloudpickle/__init__.py new file mode 100644 index 0000000000000..4e85b637800dc --- /dev/null +++ b/python/pyspark/cloudpickle/__init__.py @@ -0,0 +1,7 @@ +from __future__ import absolute_import + + +from pyspark.cloudpickle.cloudpickle import * # noqa +from pyspark.cloudpickle.cloudpickle_fast import CloudPickler, dumps, dump # noqa + +__version__ = '1.5.0' diff --git a/python/pyspark/cloudpickle/cloudpickle.py b/python/pyspark/cloudpickle/cloudpickle.py new file mode 100644 index 0000000000000..8e683e7a6988b --- /dev/null +++ b/python/pyspark/cloudpickle/cloudpickle.py @@ -0,0 +1,830 @@ +""" +This class is defined to override standard pickle functionality + +The goals of it follow: +-Serialize lambdas and nested functions to compiled byte code +-Deal with main module correctly +-Deal with other non-serializable objects + +It does not include an unpickler, as standard python unpickling suffices. + +This module was extracted from the `cloud` package, developed by `PiCloud, Inc. +`_. + +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 `PiCloud, Inc. `_. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the University of California, Berkeley nor the + names of its contributors may be used to endorse or promote + products derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +""" +from __future__ import print_function + +import builtins +import dis +import opcode +import platform +import sys +import types +import weakref +import uuid +import threading +import typing +import warnings + +from .compat import pickle +from typing import Generic, Union, Tuple, Callable +from pickle import _getattribute +from importlib._bootstrap import _find_spec + +try: # pragma: no branch + import typing_extensions as _typing_extensions + from typing_extensions import Literal, Final +except ImportError: + _typing_extensions = Literal = Final = None + +if sys.version_info >= (3, 5, 3): + from typing import ClassVar +else: # pragma: no cover + ClassVar = None + +if sys.version_info >= (3, 8): + from types import CellType +else: + def f(): + a = 1 + + def g(): + return a + return g + CellType = type(f().__closure__[0]) + + +# cloudpickle is meant for inter process communication: we expect all +# communicating processes to run the same Python version hence we favor +# communication speed over compatibility: +DEFAULT_PROTOCOL = pickle.HIGHEST_PROTOCOL + +# Track the provenance of reconstructed dynamic classes to make it possible to +# recontruct instances from the matching singleton class definition when +# appropriate and preserve the usual "isinstance" semantics of Python objects. +_DYNAMIC_CLASS_TRACKER_BY_CLASS = weakref.WeakKeyDictionary() +_DYNAMIC_CLASS_TRACKER_BY_ID = weakref.WeakValueDictionary() +_DYNAMIC_CLASS_TRACKER_LOCK = threading.Lock() + +PYPY = platform.python_implementation() == "PyPy" + +builtin_code_type = None +if PYPY: + # builtin-code objects only exist in pypy + builtin_code_type = type(float.__new__.__code__) + +_extract_code_globals_cache = weakref.WeakKeyDictionary() + + +def _get_or_create_tracker_id(class_def): + with _DYNAMIC_CLASS_TRACKER_LOCK: + class_tracker_id = _DYNAMIC_CLASS_TRACKER_BY_CLASS.get(class_def) + if class_tracker_id is None: + class_tracker_id = uuid.uuid4().hex + _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id + _DYNAMIC_CLASS_TRACKER_BY_ID[class_tracker_id] = class_def + return class_tracker_id + + +def _lookup_class_or_track(class_tracker_id, class_def): + if class_tracker_id is not None: + with _DYNAMIC_CLASS_TRACKER_LOCK: + class_def = _DYNAMIC_CLASS_TRACKER_BY_ID.setdefault( + class_tracker_id, class_def) + _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id + return class_def + + +def _whichmodule(obj, name): + """Find the module an object belongs to. + + This function differs from ``pickle.whichmodule`` in two ways: + - it does not mangle the cases where obj's module is __main__ and obj was + not found in any module. + - Errors arising during module introspection are ignored, as those errors + are considered unwanted side effects. + """ + if sys.version_info[:2] < (3, 7) and isinstance(obj, typing.TypeVar): # pragma: no branch # noqa + # Workaround bug in old Python versions: prior to Python 3.7, + # T.__module__ would always be set to "typing" even when the TypeVar T + # would be defined in a different module. + # + # For such older Python versions, we ignore the __module__ attribute of + # TypeVar instances and instead exhaustively lookup those instances in + # all currently imported modules. + module_name = None + else: + module_name = getattr(obj, '__module__', None) + + if module_name is not None: + return module_name + # Protect the iteration by using a copy of sys.modules against dynamic + # modules that trigger imports of other modules upon calls to getattr or + # other threads importing at the same time. + for module_name, module in sys.modules.copy().items(): + # Some modules such as coverage can inject non-module objects inside + # sys.modules + if ( + module_name == '__main__' or + module is None or + not isinstance(module, types.ModuleType) + ): + continue + try: + if _getattribute(module, name)[0] is obj: + return module_name + except Exception: + pass + return None + + +def _is_importable(obj, name=None): + """Dispatcher utility to test the importability of various constructs.""" + if isinstance(obj, types.FunctionType): + return _lookup_module_and_qualname(obj, name=name) is not None + elif issubclass(type(obj), type): + return _lookup_module_and_qualname(obj, name=name) is not None + elif isinstance(obj, types.ModuleType): + # We assume that sys.modules is primarily used as a cache mechanism for + # the Python import machinery. Checking if a module has been added in + # is sys.modules therefore a cheap and simple heuristic to tell us whether + # we can assume that a given module could be imported by name in + # another Python process. + return obj.__name__ in sys.modules + else: + raise TypeError( + "cannot check importability of {} instances".format( + type(obj).__name__) + ) + + +def _lookup_module_and_qualname(obj, name=None): + if name is None: + name = getattr(obj, '__qualname__', None) + if name is None: # pragma: no cover + # This used to be needed for Python 2.7 support but is probably not + # needed anymore. However we keep the __name__ introspection in case + # users of cloudpickle rely on this old behavior for unknown reasons. + name = getattr(obj, '__name__', None) + + module_name = _whichmodule(obj, name) + + if module_name is None: + # In this case, obj.__module__ is None AND obj was not found in any + # imported module. obj is thus treated as dynamic. + return None + + if module_name == "__main__": + return None + + # Note: if module_name is in sys.modules, the corresponding module is + # assumed importable at unpickling time. See #357 + module = sys.modules.get(module_name, None) + if module is None: + # The main reason why obj's module would not be imported is that this + # module has been dynamically created, using for example + # types.ModuleType. The other possibility is that module was removed + # from sys.modules after obj was created/imported. But this case is not + # supported, as the standard pickle does not support it either. + return None + + try: + obj2, parent = _getattribute(module, name) + except AttributeError: + # obj was not found inside the module it points to + return None + if obj2 is not obj: + return None + return module, name + + +def _extract_code_globals(co): + """ + Find all globals names read or written to by codeblock co + """ + out_names = _extract_code_globals_cache.get(co) + if out_names is None: + names = co.co_names + out_names = {names[oparg] for _, oparg in _walk_global_ops(co)} + + # Declaring a function inside another one using the "def ..." + # syntax generates a constant code object corresonding to the one + # of the nested function's As the nested function may itself need + # global variables, we need to introspect its code, extract its + # globals, (look for code object in it's co_consts attribute..) and + # add the result to code_globals + if co.co_consts: + for const in co.co_consts: + if isinstance(const, types.CodeType): + out_names |= _extract_code_globals(const) + + _extract_code_globals_cache[co] = out_names + + return out_names + + +def _find_imported_submodules(code, top_level_dependencies): + """ + Find currently imported submodules used by a function. + + Submodules used by a function need to be detected and referenced for the + function to work correctly at depickling time. Because submodules can be + referenced as attribute of their parent package (``package.submodule``), we + need a special introspection technique that does not rely on GLOBAL-related + opcodes to find references of them in a code object. + + Example: + ``` + import concurrent.futures + import cloudpickle + def func(): + x = concurrent.futures.ThreadPoolExecutor + if __name__ == '__main__': + cloudpickle.dumps(func) + ``` + The globals extracted by cloudpickle in the function's state include the + concurrent package, but not its submodule (here, concurrent.futures), which + is the module used by func. Find_imported_submodules will detect the usage + of concurrent.futures. Saving this module alongside with func will ensure + that calling func once depickled does not fail due to concurrent.futures + not being imported + """ + + subimports = [] + # check if any known dependency is an imported package + for x in top_level_dependencies: + if (isinstance(x, types.ModuleType) and + hasattr(x, '__package__') and x.__package__): + # check if the package has any currently loaded sub-imports + prefix = x.__name__ + '.' + # A concurrent thread could mutate sys.modules, + # make sure we iterate over a copy to avoid exceptions + for name in list(sys.modules): + # Older versions of pytest will add a "None" module to + # sys.modules. + if name is not None and name.startswith(prefix): + # check whether the function can address the sub-module + tokens = set(name[len(prefix):].split('.')) + if not tokens - set(code.co_names): + subimports.append(sys.modules[name]) + return subimports + + +def cell_set(cell, value): + """Set the value of a closure cell. + + The point of this function is to set the cell_contents attribute of a cell + after its creation. This operation is necessary in case the cell contains a + reference to the function the cell belongs to, as when calling the + function's constructor + ``f = types.FunctionType(code, globals, name, argdefs, closure)``, + closure will not be able to contain the yet-to-be-created f. + + In Python3.7, cell_contents is writeable, so setting the contents of a cell + can be done simply using + >>> cell.cell_contents = value + + In earlier Python3 versions, the cell_contents attribute of a cell is read + only, but this limitation can be worked around by leveraging the Python 3 + ``nonlocal`` keyword. + + In Python2 however, this attribute is read only, and there is no + ``nonlocal`` keyword. For this reason, we need to come up with more + complicated hacks to set this attribute. + + The chosen approach is to create a function with a STORE_DEREF opcode, + which sets the content of a closure variable. Typically: + + >>> def inner(value): + ... lambda: cell # the lambda makes cell a closure + ... cell = value # cell is a closure, so this triggers a STORE_DEREF + + (Note that in Python2, A STORE_DEREF can never be triggered from an inner + function. The function g for example here + >>> def f(var): + ... def g(): + ... var += 1 + ... return g + + will not modify the closure variable ``var```inplace, but instead try to + load a local variable var and increment it. As g does not assign the local + variable ``var`` any initial value, calling f(1)() will fail at runtime.) + + Our objective is to set the value of a given cell ``cell``. So we need to + somewhat reference our ``cell`` object into the ``inner`` function so that + this object (and not the smoke cell of the lambda function) gets affected + by the STORE_DEREF operation. + + In inner, ``cell`` is referenced as a cell variable (an enclosing variable + that is referenced by the inner function). If we create a new function + cell_set with the exact same code as ``inner``, but with ``cell`` marked as + a free variable instead, the STORE_DEREF will be applied on its closure - + ``cell``, which we can specify explicitly during construction! The new + cell_set variable thus actually sets the contents of a specified cell! + + Note: we do not make use of the ``nonlocal`` keyword to set the contents of + a cell in early python3 versions to limit possible syntax errors in case + test and checker libraries decide to parse the whole file. + """ + + if sys.version_info[:2] >= (3, 7): # pragma: no branch + cell.cell_contents = value + else: + _cell_set = types.FunctionType( + _cell_set_template_code, {}, '_cell_set', (), (cell,),) + _cell_set(value) + + +def _make_cell_set_template_code(): + def _cell_set_factory(value): + lambda: cell + cell = value + + co = _cell_set_factory.__code__ + + _cell_set_template_code = types.CodeType( + co.co_argcount, + co.co_kwonlyargcount, # Python 3 only argument + co.co_nlocals, + co.co_stacksize, + co.co_flags, + co.co_code, + co.co_consts, + co.co_names, + co.co_varnames, + co.co_filename, + co.co_name, + co.co_firstlineno, + co.co_lnotab, + co.co_cellvars, # co_freevars is initialized with co_cellvars + (), # co_cellvars is made empty + ) + return _cell_set_template_code + + +if sys.version_info[:2] < (3, 7): + _cell_set_template_code = _make_cell_set_template_code() + +# relevant opcodes +STORE_GLOBAL = opcode.opmap['STORE_GLOBAL'] +DELETE_GLOBAL = opcode.opmap['DELETE_GLOBAL'] +LOAD_GLOBAL = opcode.opmap['LOAD_GLOBAL'] +GLOBAL_OPS = (STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL) +HAVE_ARGUMENT = dis.HAVE_ARGUMENT +EXTENDED_ARG = dis.EXTENDED_ARG + + +_BUILTIN_TYPE_NAMES = {} +for k, v in types.__dict__.items(): + if type(v) is type: + _BUILTIN_TYPE_NAMES[v] = k + + +def _builtin_type(name): + if name == "ClassType": # pragma: no cover + # Backward compat to load pickle files generated with cloudpickle + # < 1.3 even if loading pickle files from older versions is not + # officially supported. + return type + return getattr(types, name) + + +def _walk_global_ops(code): + """ + Yield (opcode, argument number) tuples for all + global-referencing instructions in *code*. + """ + for instr in dis.get_instructions(code): + op = instr.opcode + if op in GLOBAL_OPS: + yield op, instr.arg + + +def _extract_class_dict(cls): + """Retrieve a copy of the dict of a class without the inherited methods""" + clsdict = dict(cls.__dict__) # copy dict proxy to a dict + if len(cls.__bases__) == 1: + inherited_dict = cls.__bases__[0].__dict__ + else: + inherited_dict = {} + for base in reversed(cls.__bases__): + inherited_dict.update(base.__dict__) + to_remove = [] + for name, value in clsdict.items(): + try: + base_value = inherited_dict[name] + if value is base_value: + to_remove.append(name) + except KeyError: + pass + for name in to_remove: + clsdict.pop(name) + return clsdict + + +if sys.version_info[:2] < (3, 7): # pragma: no branch + def _is_parametrized_type_hint(obj): + # This is very cheap but might generate false positives. + # general typing Constructs + is_typing = getattr(obj, '__origin__', None) is not None + + # typing_extensions.Literal + is_litteral = getattr(obj, '__values__', None) is not None + + # typing_extensions.Final + is_final = getattr(obj, '__type__', None) is not None + + # typing.Union/Tuple for old Python 3.5 + is_union = getattr(obj, '__union_params__', None) is not None + is_tuple = getattr(obj, '__tuple_params__', None) is not None + is_callable = ( + getattr(obj, '__result__', None) is not None and + getattr(obj, '__args__', None) is not None + ) + return any((is_typing, is_litteral, is_final, is_union, is_tuple, + is_callable)) + + def _create_parametrized_type_hint(origin, args): + return origin[args] +else: + _is_parametrized_type_hint = None + _create_parametrized_type_hint = None + + +def parametrized_type_hint_getinitargs(obj): + # The distorted type check sematic for typing construct becomes: + # ``type(obj) is type(TypeHint)``, which means "obj is a + # parametrized TypeHint" + if type(obj) is type(Literal): # pragma: no branch + initargs = (Literal, obj.__values__) + elif type(obj) is type(Final): # pragma: no branch + initargs = (Final, obj.__type__) + elif type(obj) is type(ClassVar): + initargs = (ClassVar, obj.__type__) + elif type(obj) is type(Generic): + parameters = obj.__parameters__ + if len(obj.__parameters__) > 0: + # in early Python 3.5, __parameters__ was sometimes + # preferred to __args__ + initargs = (obj.__origin__, parameters) + + else: + initargs = (obj.__origin__, obj.__args__) + elif type(obj) is type(Union): + if sys.version_info < (3, 5, 3): # pragma: no cover + initargs = (Union, obj.__union_params__) + else: + initargs = (Union, obj.__args__) + elif type(obj) is type(Tuple): + if sys.version_info < (3, 5, 3): # pragma: no cover + initargs = (Tuple, obj.__tuple_params__) + else: + initargs = (Tuple, obj.__args__) + elif type(obj) is type(Callable): + if sys.version_info < (3, 5, 3): # pragma: no cover + args = obj.__args__ + result = obj.__result__ + if args != Ellipsis: + if isinstance(args, tuple): + args = list(args) + else: + args = [args] + else: + (*args, result) = obj.__args__ + if len(args) == 1 and args[0] is Ellipsis: + args = Ellipsis + else: + args = list(args) + initargs = (Callable, (args, result)) + else: # pragma: no cover + raise pickle.PicklingError( + "Cloudpickle Error: Unknown type {}".format(type(obj)) + ) + return initargs + + +# Tornado support + +def is_tornado_coroutine(func): + """ + Return whether *func* is a Tornado coroutine function. + Running coroutines are not supported. + """ + if 'tornado.gen' not in sys.modules: + return False + gen = sys.modules['tornado.gen'] + if not hasattr(gen, "is_coroutine_function"): + # Tornado version is too old + return False + return gen.is_coroutine_function(func) + + +def _rebuild_tornado_coroutine(func): + from tornado import gen + return gen.coroutine(func) + + +# including pickles unloading functions in this namespace +load = pickle.load +loads = pickle.loads + + +# hack for __import__ not working as desired +def subimport(name): + __import__(name) + return sys.modules[name] + + +def dynamic_subimport(name, vars): + mod = types.ModuleType(name) + mod.__dict__.update(vars) + mod.__dict__['__builtins__'] = builtins.__dict__ + return mod + + +def _gen_ellipsis(): + return Ellipsis + + +def _gen_not_implemented(): + return NotImplemented + + +def _get_cell_contents(cell): + try: + return cell.cell_contents + except ValueError: + # sentinel used by ``_fill_function`` which will leave the cell empty + return _empty_cell_value + + +def instance(cls): + """Create a new instance of a class. + + Parameters + ---------- + cls : type + The class to create an instance of. + + Returns + ------- + instance : cls + A new instance of ``cls``. + """ + return cls() + + +@instance +class _empty_cell_value(object): + """sentinel for empty closures + """ + @classmethod + def __reduce__(cls): + return cls.__name__ + + +def _fill_function(*args): + """Fills in the rest of function data into the skeleton function object + + The skeleton itself is create by _make_skel_func(). + """ + if len(args) == 2: + func = args[0] + state = args[1] + elif len(args) == 5: + # Backwards compat for cloudpickle v0.4.0, after which the `module` + # argument was introduced + func = args[0] + keys = ['globals', 'defaults', 'dict', 'closure_values'] + state = dict(zip(keys, args[1:])) + elif len(args) == 6: + # Backwards compat for cloudpickle v0.4.1, after which the function + # state was passed as a dict to the _fill_function it-self. + func = args[0] + keys = ['globals', 'defaults', 'dict', 'module', 'closure_values'] + state = dict(zip(keys, args[1:])) + else: + raise ValueError('Unexpected _fill_value arguments: %r' % (args,)) + + # - At pickling time, any dynamic global variable used by func is + # serialized by value (in state['globals']). + # - At unpickling time, func's __globals__ attribute is initialized by + # first retrieving an empty isolated namespace that will be shared + # with other functions pickled from the same original module + # by the same CloudPickler instance and then updated with the + # content of state['globals'] to populate the shared isolated + # namespace with all the global variables that are specifically + # referenced for this function. + func.__globals__.update(state['globals']) + + func.__defaults__ = state['defaults'] + func.__dict__ = state['dict'] + if 'annotations' in state: + func.__annotations__ = state['annotations'] + if 'doc' in state: + func.__doc__ = state['doc'] + if 'name' in state: + func.__name__ = state['name'] + if 'module' in state: + func.__module__ = state['module'] + if 'qualname' in state: + func.__qualname__ = state['qualname'] + if 'kwdefaults' in state: + func.__kwdefaults__ = state['kwdefaults'] + # _cloudpickle_subimports is a set of submodules that must be loaded for + # the pickled function to work correctly at unpickling time. Now that these + # submodules are depickled (hence imported), they can be removed from the + # object's state (the object state only served as a reference holder to + # these submodules) + if '_cloudpickle_submodules' in state: + state.pop('_cloudpickle_submodules') + + cells = func.__closure__ + if cells is not None: + for cell, value in zip(cells, state['closure_values']): + if value is not _empty_cell_value: + cell_set(cell, value) + + return func + + +def _make_empty_cell(): + if False: + # trick the compiler into creating an empty cell in our lambda + cell = None + raise AssertionError('this route should not be executed') + + return (lambda: cell).__closure__[0] + + +def _make_cell(value=_empty_cell_value): + cell = _make_empty_cell() + if value is not _empty_cell_value: + cell_set(cell, value) + return cell + + +def _make_skel_func(code, cell_count, base_globals=None): + """ Creates a skeleton function object that contains just the provided + code and the correct number of cells in func_closure. All other + func attributes (e.g. func_globals) are empty. + """ + # This function is deprecated and should be removed in cloudpickle 1.7 + warnings.warn( + "A pickle file created using an old (<=1.4.1) version of cloudpicke " + "is currently being loaded. This is not supported by cloudpickle and " + "will break in cloudpickle 1.7", category=UserWarning + ) + # This is backward-compatibility code: for cloudpickle versions between + # 0.5.4 and 0.7, base_globals could be a string or None. base_globals + # should now always be a dictionary. + if base_globals is None or isinstance(base_globals, str): + base_globals = {} + + base_globals['__builtins__'] = __builtins__ + + closure = ( + tuple(_make_empty_cell() for _ in range(cell_count)) + if cell_count >= 0 else + None + ) + return types.FunctionType(code, base_globals, None, None, closure) + + +def _make_skeleton_class(type_constructor, name, bases, type_kwargs, + class_tracker_id, extra): + """Build dynamic class with an empty __dict__ to be filled once memoized + + If class_tracker_id is not None, try to lookup an existing class definition + matching that id. If none is found, track a newly reconstructed class + definition under that id so that other instances stemming from the same + class id will also reuse this class definition. + + The "extra" variable is meant to be a dict (or None) that can be used for + forward compatibility shall the need arise. + """ + skeleton_class = types.new_class( + name, bases, {'metaclass': type_constructor}, + lambda ns: ns.update(type_kwargs) + ) + return _lookup_class_or_track(class_tracker_id, skeleton_class) + + +def _rehydrate_skeleton_class(skeleton_class, class_dict): + """Put attributes from `class_dict` back on `skeleton_class`. + + See CloudPickler.save_dynamic_class for more info. + """ + registry = None + for attrname, attr in class_dict.items(): + if attrname == "_abc_impl": + registry = attr + else: + setattr(skeleton_class, attrname, attr) + if registry is not None: + for subclass in registry: + skeleton_class.register(subclass) + + return skeleton_class + + +def _make_skeleton_enum(bases, name, qualname, members, module, + class_tracker_id, extra): + """Build dynamic enum with an empty __dict__ to be filled once memoized + + The creation of the enum class is inspired by the code of + EnumMeta._create_. + + If class_tracker_id is not None, try to lookup an existing enum definition + matching that id. If none is found, track a newly reconstructed enum + definition under that id so that other instances stemming from the same + class id will also reuse this enum definition. + + The "extra" variable is meant to be a dict (or None) that can be used for + forward compatibility shall the need arise. + """ + # enums always inherit from their base Enum class at the last position in + # the list of base classes: + enum_base = bases[-1] + metacls = enum_base.__class__ + classdict = metacls.__prepare__(name, bases) + + for member_name, member_value in members.items(): + classdict[member_name] = member_value + enum_class = metacls.__new__(metacls, name, bases, classdict) + enum_class.__module__ = module + enum_class.__qualname__ = qualname + + return _lookup_class_or_track(class_tracker_id, enum_class) + + +def _make_typevar(name, bound, constraints, covariant, contravariant, + class_tracker_id): + tv = typing.TypeVar( + name, *constraints, bound=bound, + covariant=covariant, contravariant=contravariant + ) + if class_tracker_id is not None: + return _lookup_class_or_track(class_tracker_id, tv) + else: # pragma: nocover + # Only for Python 3.5.3 compat. + return tv + + +def _decompose_typevar(obj): + try: + class_tracker_id = _get_or_create_tracker_id(obj) + except TypeError: # pragma: nocover + # TypeVar instances are not weakref-able in Python 3.5.3 + class_tracker_id = None + return ( + obj.__name__, obj.__bound__, obj.__constraints__, + obj.__covariant__, obj.__contravariant__, + class_tracker_id, + ) + + +def _typevar_reduce(obj): + # TypeVar instances have no __qualname__ hence we pass the name explicitly. + module_and_name = _lookup_module_and_qualname(obj, name=obj.__name__) + if module_and_name is None: + return (_make_typevar, _decompose_typevar(obj)) + return (getattr, module_and_name) + + +def _get_bases(typ): + if hasattr(typ, '__orig_bases__'): + # For generic types (see PEP 560) + bases_attr = '__orig_bases__' + else: + # For regular class objects + bases_attr = '__bases__' + return getattr(typ, bases_attr) diff --git a/python/pyspark/cloudpickle/cloudpickle_fast.py b/python/pyspark/cloudpickle/cloudpickle_fast.py new file mode 100644 index 0000000000000..e8e46b88fdc91 --- /dev/null +++ b/python/pyspark/cloudpickle/cloudpickle_fast.py @@ -0,0 +1,747 @@ +""" +New, fast version of the CloudPickler. + +This new CloudPickler class can now extend the fast C Pickler instead of the +previous Python implementation of the Pickler class. Because this functionality +is only available for Python versions 3.8+, a lot of backward-compatibility +code is also removed. + +Note that the C Pickler sublassing API is CPython-specific. Therefore, some +guards present in cloudpickle.py that were written to handle PyPy specificities +are not present in cloudpickle_fast.py +""" +import abc +import copyreg +import io +import itertools +import logging +import sys +import struct +import types +import weakref +import typing + +from enum import Enum +from collections import ChainMap + +from .compat import pickle, Pickler +from .cloudpickle import ( + _extract_code_globals, _BUILTIN_TYPE_NAMES, DEFAULT_PROTOCOL, + _find_imported_submodules, _get_cell_contents, _is_importable, + _builtin_type, _get_or_create_tracker_id, _make_skeleton_class, + _make_skeleton_enum, _extract_class_dict, dynamic_subimport, subimport, + _typevar_reduce, _get_bases, _make_cell, _make_empty_cell, CellType, + _is_parametrized_type_hint, PYPY, cell_set, + parametrized_type_hint_getinitargs, _create_parametrized_type_hint, + builtin_code_type + +) + + +if pickle.HIGHEST_PROTOCOL >= 5 and not PYPY: + # Shorthands similar to pickle.dump/pickle.dumps + + def dump(obj, file, protocol=None, buffer_callback=None): + """Serialize obj as bytes streamed into file + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + CloudPickler( + file, protocol=protocol, buffer_callback=buffer_callback + ).dump(obj) + + def dumps(obj, protocol=None, buffer_callback=None): + """Serialize obj as a string of bytes allocated in memory + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + with io.BytesIO() as file: + cp = CloudPickler( + file, protocol=protocol, buffer_callback=buffer_callback + ) + cp.dump(obj) + return file.getvalue() + +else: + # Shorthands similar to pickle.dump/pickle.dumps + def dump(obj, file, protocol=None): + """Serialize obj as bytes streamed into file + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + CloudPickler(file, protocol=protocol).dump(obj) + + def dumps(obj, protocol=None): + """Serialize obj as a string of bytes allocated in memory + + protocol defaults to cloudpickle.DEFAULT_PROTOCOL which is an alias to + pickle.HIGHEST_PROTOCOL. This setting favors maximum communication + speed between processes running the same Python version. + + Set protocol=pickle.DEFAULT_PROTOCOL instead if you need to ensure + compatibility with older versions of Python. + """ + with io.BytesIO() as file: + cp = CloudPickler(file, protocol=protocol) + cp.dump(obj) + return file.getvalue() + + +load, loads = pickle.load, pickle.loads + + +# COLLECTION OF OBJECTS __getnewargs__-LIKE METHODS +# ------------------------------------------------- + +def _class_getnewargs(obj): + type_kwargs = {} + if "__slots__" in obj.__dict__: + type_kwargs["__slots__"] = obj.__slots__ + + __dict__ = obj.__dict__.get('__dict__', None) + if isinstance(__dict__, property): + type_kwargs['__dict__'] = __dict__ + + return (type(obj), obj.__name__, _get_bases(obj), type_kwargs, + _get_or_create_tracker_id(obj), None) + + +def _enum_getnewargs(obj): + members = dict((e.name, e.value) for e in obj) + return (obj.__bases__, obj.__name__, obj.__qualname__, members, + obj.__module__, _get_or_create_tracker_id(obj), None) + + +# COLLECTION OF OBJECTS RECONSTRUCTORS +# ------------------------------------ +def _file_reconstructor(retval): + return retval + + +# COLLECTION OF OBJECTS STATE GETTERS +# ----------------------------------- +def _function_getstate(func): + # - Put func's dynamic attributes (stored in func.__dict__) in state. These + # attributes will be restored at unpickling time using + # f.__dict__.update(state) + # - Put func's members into slotstate. Such attributes will be restored at + # unpickling time by iterating over slotstate and calling setattr(func, + # slotname, slotvalue) + slotstate = { + "__name__": func.__name__, + "__qualname__": func.__qualname__, + "__annotations__": func.__annotations__, + "__kwdefaults__": func.__kwdefaults__, + "__defaults__": func.__defaults__, + "__module__": func.__module__, + "__doc__": func.__doc__, + "__closure__": func.__closure__, + } + + f_globals_ref = _extract_code_globals(func.__code__) + f_globals = {k: func.__globals__[k] for k in f_globals_ref if k in + func.__globals__} + + closure_values = ( + list(map(_get_cell_contents, func.__closure__)) + if func.__closure__ is not None else () + ) + + # Extract currently-imported submodules used by func. Storing these modules + # in a smoke _cloudpickle_subimports attribute of the object's state will + # trigger the side effect of importing these modules at unpickling time + # (which is necessary for func to work correctly once depickled) + slotstate["_cloudpickle_submodules"] = _find_imported_submodules( + func.__code__, itertools.chain(f_globals.values(), closure_values)) + slotstate["__globals__"] = f_globals + + state = func.__dict__ + return state, slotstate + + +def _class_getstate(obj): + clsdict = _extract_class_dict(obj) + clsdict.pop('__weakref__', None) + + if issubclass(type(obj), abc.ABCMeta): + # If obj is an instance of an ABCMeta subclass, dont pickle the + # cache/negative caches populated during isinstance/issubclass + # checks, but pickle the list of registered subclasses of obj. + clsdict.pop('_abc_cache', None) + clsdict.pop('_abc_negative_cache', None) + clsdict.pop('_abc_negative_cache_version', None) + registry = clsdict.pop('_abc_registry', None) + if registry is None: + # in Python3.7+, the abc caches and registered subclasses of a + # class are bundled into the single _abc_impl attribute + clsdict.pop('_abc_impl', None) + (registry, _, _, _) = abc._get_dump(obj) + + clsdict["_abc_impl"] = [subclass_weakref() + for subclass_weakref in registry] + else: + # In the above if clause, registry is a set of weakrefs -- in + # this case, registry is a WeakSet + clsdict["_abc_impl"] = [type_ for type_ in registry] + + if "__slots__" in clsdict: + # pickle string length optimization: member descriptors of obj are + # created automatically from obj's __slots__ attribute, no need to + # save them in obj's state + if isinstance(obj.__slots__, str): + clsdict.pop(obj.__slots__) + else: + for k in obj.__slots__: + clsdict.pop(k, None) + + clsdict.pop('__dict__', None) # unpicklable property object + + return (clsdict, {}) + + +def _enum_getstate(obj): + clsdict, slotstate = _class_getstate(obj) + + members = dict((e.name, e.value) for e in obj) + # Cleanup the clsdict that will be passed to _rehydrate_skeleton_class: + # Those attributes are already handled by the metaclass. + for attrname in ["_generate_next_value_", "_member_names_", + "_member_map_", "_member_type_", + "_value2member_map_"]: + clsdict.pop(attrname, None) + for member in members: + clsdict.pop(member) + # Special handling of Enum subclasses + return clsdict, slotstate + + +# COLLECTIONS OF OBJECTS REDUCERS +# ------------------------------- +# A reducer is a function taking a single argument (obj), and that returns a +# tuple with all the necessary data to re-construct obj. Apart from a few +# exceptions (list, dict, bytes, int, etc.), a reducer is necessary to +# correctly pickle an object. +# While many built-in objects (Exceptions objects, instances of the "object" +# class, etc), are shipped with their own built-in reducer (invoked using +# obj.__reduce__), some do not. The following methods were created to "fill +# these holes". + +def _code_reduce(obj): + """codeobject reducer""" + if hasattr(obj, "co_posonlyargcount"): # pragma: no branch + args = ( + obj.co_argcount, obj.co_posonlyargcount, + obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, + obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, + obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, + obj.co_cellvars + ) + else: + args = ( + obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, + obj.co_stacksize, obj.co_flags, obj.co_code, obj.co_consts, + obj.co_names, obj.co_varnames, obj.co_filename, + obj.co_name, obj.co_firstlineno, obj.co_lnotab, + obj.co_freevars, obj.co_cellvars + ) + return types.CodeType, args + + +def _cell_reduce(obj): + """Cell (containing values of a function's free variables) reducer""" + try: + obj.cell_contents + except ValueError: # cell is empty + return _make_empty_cell, () + else: + return _make_cell, (obj.cell_contents, ) + + +def _classmethod_reduce(obj): + orig_func = obj.__func__ + return type(obj), (orig_func,) + + +def _file_reduce(obj): + """Save a file""" + import io + + if not hasattr(obj, "name") or not hasattr(obj, "mode"): + raise pickle.PicklingError( + "Cannot pickle files that do not map to an actual file" + ) + if obj is sys.stdout: + return getattr, (sys, "stdout") + if obj is sys.stderr: + return getattr, (sys, "stderr") + if obj is sys.stdin: + raise pickle.PicklingError("Cannot pickle standard input") + if obj.closed: + raise pickle.PicklingError("Cannot pickle closed files") + if hasattr(obj, "isatty") and obj.isatty(): + raise pickle.PicklingError( + "Cannot pickle files that map to tty objects" + ) + if "r" not in obj.mode and "+" not in obj.mode: + raise pickle.PicklingError( + "Cannot pickle files that are not opened for reading: %s" + % obj.mode + ) + + name = obj.name + + retval = io.StringIO() + + try: + # Read the whole file + curloc = obj.tell() + obj.seek(0) + contents = obj.read() + obj.seek(curloc) + except IOError as e: + raise pickle.PicklingError( + "Cannot pickle file %s as it cannot be read" % name + ) from e + retval.write(contents) + retval.seek(curloc) + + retval.name = name + return _file_reconstructor, (retval,) + + +def _getset_descriptor_reduce(obj): + return getattr, (obj.__objclass__, obj.__name__) + + +def _mappingproxy_reduce(obj): + return types.MappingProxyType, (dict(obj),) + + +def _memoryview_reduce(obj): + return bytes, (obj.tobytes(),) + + +def _module_reduce(obj): + if _is_importable(obj): + return subimport, (obj.__name__,) + else: + obj.__dict__.pop('__builtins__', None) + return dynamic_subimport, (obj.__name__, vars(obj)) + + +def _method_reduce(obj): + return (types.MethodType, (obj.__func__, obj.__self__)) + + +def _logger_reduce(obj): + return logging.getLogger, (obj.name,) + + +def _root_logger_reduce(obj): + return logging.getLogger, () + + +def _property_reduce(obj): + return property, (obj.fget, obj.fset, obj.fdel, obj.__doc__) + + +def _weakset_reduce(obj): + return weakref.WeakSet, (list(obj),) + + +def _dynamic_class_reduce(obj): + """ + Save a class that can't be stored as module global. + + This method is used to serialize classes that are defined inside + functions, or that otherwise can't be serialized as attribute lookups + from global modules. + """ + if Enum is not None and issubclass(obj, Enum): + return ( + _make_skeleton_enum, _enum_getnewargs(obj), _enum_getstate(obj), + None, None, _class_setstate + ) + else: + return ( + _make_skeleton_class, _class_getnewargs(obj), _class_getstate(obj), + None, None, _class_setstate + ) + + +def _class_reduce(obj): + """Select the reducer depending on the dynamic nature of the class obj""" + if obj is type(None): # noqa + return type, (None,) + elif obj is type(Ellipsis): + return type, (Ellipsis,) + elif obj is type(NotImplemented): + return type, (NotImplemented,) + elif obj in _BUILTIN_TYPE_NAMES: + return _builtin_type, (_BUILTIN_TYPE_NAMES[obj],) + elif not _is_importable(obj): + return _dynamic_class_reduce(obj) + return NotImplemented + + +# COLLECTIONS OF OBJECTS STATE SETTERS +# ------------------------------------ +# state setters are called at unpickling time, once the object is created and +# it has to be updated to how it was at unpickling time. + + +def _function_setstate(obj, state): + """Update the state of a dynaamic function. + + As __closure__ and __globals__ are readonly attributes of a function, we + cannot rely on the native setstate routine of pickle.load_build, that calls + setattr on items of the slotstate. Instead, we have to modify them inplace. + """ + state, slotstate = state + obj.__dict__.update(state) + + obj_globals = slotstate.pop("__globals__") + obj_closure = slotstate.pop("__closure__") + # _cloudpickle_subimports is a set of submodules that must be loaded for + # the pickled function to work correctly at unpickling time. Now that these + # submodules are depickled (hence imported), they can be removed from the + # object's state (the object state only served as a reference holder to + # these submodules) + slotstate.pop("_cloudpickle_submodules") + + obj.__globals__.update(obj_globals) + obj.__globals__["__builtins__"] = __builtins__ + + if obj_closure is not None: + for i, cell in enumerate(obj_closure): + try: + value = cell.cell_contents + except ValueError: # cell is empty + continue + cell_set(obj.__closure__[i], value) + + for k, v in slotstate.items(): + setattr(obj, k, v) + + +def _class_setstate(obj, state): + state, slotstate = state + registry = None + for attrname, attr in state.items(): + if attrname == "_abc_impl": + registry = attr + else: + setattr(obj, attrname, attr) + if registry is not None: + for subclass in registry: + obj.register(subclass) + + return obj + + +class CloudPickler(Pickler): + # set of reducers defined and used by cloudpickle (private) + _dispatch_table = {} + _dispatch_table[classmethod] = _classmethod_reduce + _dispatch_table[io.TextIOWrapper] = _file_reduce + _dispatch_table[logging.Logger] = _logger_reduce + _dispatch_table[logging.RootLogger] = _root_logger_reduce + _dispatch_table[memoryview] = _memoryview_reduce + _dispatch_table[property] = _property_reduce + _dispatch_table[staticmethod] = _classmethod_reduce + _dispatch_table[CellType] = _cell_reduce + _dispatch_table[types.CodeType] = _code_reduce + _dispatch_table[types.GetSetDescriptorType] = _getset_descriptor_reduce + _dispatch_table[types.ModuleType] = _module_reduce + _dispatch_table[types.MethodType] = _method_reduce + _dispatch_table[types.MappingProxyType] = _mappingproxy_reduce + _dispatch_table[weakref.WeakSet] = _weakset_reduce + _dispatch_table[typing.TypeVar] = _typevar_reduce + + dispatch_table = ChainMap(_dispatch_table, copyreg.dispatch_table) + + # function reducers are defined as instance methods of CloudPickler + # objects, as they rely on a CloudPickler attribute (globals_ref) + def _dynamic_function_reduce(self, func): + """Reduce a function that is not pickleable via attribute lookup.""" + newargs = self._function_getnewargs(func) + state = _function_getstate(func) + return (types.FunctionType, newargs, state, None, None, + _function_setstate) + + def _function_reduce(self, obj): + """Reducer for function objects. + + If obj is a top-level attribute of a file-backed module, this + reducer returns NotImplemented, making the CloudPickler fallback to + traditional _pickle.Pickler routines to save obj. Otherwise, it reduces + obj using a custom cloudpickle reducer designed specifically to handle + dynamic functions. + + As opposed to cloudpickle.py, There no special handling for builtin + pypy functions because cloudpickle_fast is CPython-specific. + """ + if _is_importable(obj): + return NotImplemented + else: + return self._dynamic_function_reduce(obj) + + def _function_getnewargs(self, func): + code = func.__code__ + + # base_globals represents the future global namespace of func at + # unpickling time. Looking it up and storing it in + # CloudpiPickler.globals_ref allow functions sharing the same globals + # at pickling time to also share them once unpickled, at one condition: + # since globals_ref is an attribute of a CloudPickler instance, and + # that a new CloudPickler is created each time pickle.dump or + # pickle.dumps is called, functions also need to be saved within the + # same invocation of cloudpickle.dump/cloudpickle.dumps (for example: + # cloudpickle.dumps([f1, f2])). There is no such limitation when using + # CloudPickler.dump, as long as the multiple invocations are bound to + # the same CloudPickler. + base_globals = self.globals_ref.setdefault(id(func.__globals__), {}) + + if base_globals == {}: + # Add module attributes used to resolve relative imports + # instructions inside func. + for k in ["__package__", "__name__", "__path__", "__file__"]: + if k in func.__globals__: + base_globals[k] = func.__globals__[k] + + # Do not bind the free variables before the function is created to + # avoid infinite recursion. + if func.__closure__ is None: + closure = None + else: + closure = tuple( + _make_empty_cell() for _ in range(len(code.co_freevars))) + + return code, base_globals, None, None, closure + + def dump(self, obj): + try: + return Pickler.dump(self, obj) + except RuntimeError as e: + if "recursion" in e.args[0]: + msg = ( + "Could not pickle object as excessively deep recursion " + "required." + ) + raise pickle.PicklingError(msg) from e + else: + raise + + if pickle.HIGHEST_PROTOCOL >= 5: + # `CloudPickler.dispatch` is only left for backward compatibility - note + # that when using protocol 5, `CloudPickler.dispatch` is not an + # extension of `Pickler.dispatch` dictionary, because CloudPickler + # subclasses the C-implemented Pickler, which does not expose a + # `dispatch` attribute. Earlier versions of the protocol 5 CloudPickler + # used `CloudPickler.dispatch` as a class-level attribute storing all + # reducers implemented by cloudpickle, but the attribute name was not a + # great choice given the meaning of `Cloudpickler.dispatch` when + # `CloudPickler` extends the pure-python pickler. + dispatch = dispatch_table + + # Implementation of the reducer_override callback, in order to + # efficiently serialize dynamic functions and classes by subclassing + # the C-implemented Pickler. + # TODO: decorrelate reducer_override (which is tied to CPython's + # implementation - would it make sense to backport it to pypy? - and + # pickle's protocol 5 which is implementation agnostic. Currently, the + # availability of both notions coincide on CPython's pickle and the + # pickle5 backport, but it may not be the case anymore when pypy + # implements protocol 5 + def __init__(self, file, protocol=None, buffer_callback=None): + if protocol is None: + protocol = DEFAULT_PROTOCOL + Pickler.__init__( + self, file, protocol=protocol, buffer_callback=buffer_callback + ) + # map functions __globals__ attribute ids, to ensure that functions + # sharing the same global namespace at pickling time also share + # their global namespace at unpickling time. + self.globals_ref = {} + self.proto = int(protocol) + + def reducer_override(self, obj): + """Type-agnostic reducing callback for function and classes. + + For performance reasons, subclasses of the C _pickle.Pickler class + cannot register custom reducers for functions and classes in the + dispatch_table. Reducer for such types must instead implemented in + the special reducer_override method. + + Note that method will be called for any object except a few + builtin-types (int, lists, dicts etc.), which differs from reducers + in the Pickler's dispatch_table, each of them being invoked for + objects of a specific type only. + + This property comes in handy for classes: although most classes are + instances of the ``type`` metaclass, some of them can be instances + of other custom metaclasses (such as enum.EnumMeta for example). In + particular, the metaclass will likely not be known in advance, and + thus cannot be special-cased using an entry in the dispatch_table. + reducer_override, among other things, allows us to register a + reducer that will be called for any class, independently of its + type. + + + Notes: + + * reducer_override has the priority over dispatch_table-registered + reducers. + * reducer_override can be used to fix other limitations of + cloudpickle for other types that suffered from type-specific + reducers, such as Exceptions. See + https://github.com/cloudpipe/cloudpickle/issues/248 + """ + if sys.version_info[:2] < (3, 7) and _is_parametrized_type_hint(obj): # noqa # pragma: no branch + return ( + _create_parametrized_type_hint, + parametrized_type_hint_getinitargs(obj) + ) + t = type(obj) + try: + is_anyclass = issubclass(t, type) + except TypeError: # t is not a class (old Boost; see SF #502085) + is_anyclass = False + + if is_anyclass: + return _class_reduce(obj) + elif isinstance(obj, types.FunctionType): + return self._function_reduce(obj) + else: + # fallback to save_global, including the Pickler's + # distpatch_table + return NotImplemented + + else: + # When reducer_override is not available, hack the pure-Python + # Pickler's types.FunctionType and type savers. Note: the type saver + # must override Pickler.save_global, because pickle.py contains a + # hard-coded call to save_global when pickling meta-classes. + dispatch = Pickler.dispatch.copy() + + def __init__(self, file, protocol=None): + if protocol is None: + protocol = DEFAULT_PROTOCOL + Pickler.__init__(self, file, protocol=protocol) + # map functions __globals__ attribute ids, to ensure that functions + # sharing the same global namespace at pickling time also share + # their global namespace at unpickling time. + self.globals_ref = {} + assert hasattr(self, 'proto') + + def _save_reduce_pickle5(self, func, args, state=None, listitems=None, + dictitems=None, state_setter=None, obj=None): + save = self.save + write = self.write + self.save_reduce( + func, args, state=None, listitems=listitems, + dictitems=dictitems, obj=obj + ) + # backport of the Python 3.8 state_setter pickle operations + save(state_setter) + save(obj) # simple BINGET opcode as obj is already memoized. + save(state) + write(pickle.TUPLE2) + # Trigger a state_setter(obj, state) function call. + write(pickle.REDUCE) + # The purpose of state_setter is to carry-out an + # inplace modification of obj. We do not care about what the + # method might return, so its output is eventually removed from + # the stack. + write(pickle.POP) + + def save_global(self, obj, name=None, pack=struct.pack): + """ + Save a "global". + + The name of this method is somewhat misleading: all types get + dispatched here. + """ + if obj is type(None): # noqa + return self.save_reduce(type, (None,), obj=obj) + elif obj is type(Ellipsis): + return self.save_reduce(type, (Ellipsis,), obj=obj) + elif obj is type(NotImplemented): + return self.save_reduce(type, (NotImplemented,), obj=obj) + elif obj in _BUILTIN_TYPE_NAMES: + return self.save_reduce( + _builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) + + if sys.version_info[:2] < (3, 7) and _is_parametrized_type_hint(obj): # noqa # pragma: no branch + # Parametrized typing constructs in Python < 3.7 are not + # compatible with type checks and ``isinstance`` semantics. For + # this reason, it is easier to detect them using a + # duck-typing-based check (``_is_parametrized_type_hint``) than + # to populate the Pickler's dispatch with type-specific savers. + self.save_reduce( + _create_parametrized_type_hint, + parametrized_type_hint_getinitargs(obj), + obj=obj + ) + elif name is not None: + Pickler.save_global(self, obj, name=name) + elif not _is_importable(obj, name=name): + self._save_reduce_pickle5(*_dynamic_class_reduce(obj), obj=obj) + else: + Pickler.save_global(self, obj, name=name) + dispatch[type] = save_global + + def save_function(self, obj, name=None): + """ Registered with the dispatch to handle all function types. + + Determines what kind of function obj is (e.g. lambda, defined at + interactive prompt, etc) and handles the pickling appropriately. + """ + if _is_importable(obj, name=name): + return Pickler.save_global(self, obj, name=name) + elif PYPY and isinstance(obj.__code__, builtin_code_type): + return self.save_pypy_builtin_func(obj) + else: + return self._save_reduce_pickle5( + *self._dynamic_function_reduce(obj), obj=obj + ) + + def save_pypy_builtin_func(self, obj): + """Save pypy equivalent of builtin functions. + PyPy does not have the concept of builtin-functions. Instead, + builtin-functions are simple function instances, but with a + builtin-code attribute. + Most of the time, builtin functions should be pickled by attribute. + But PyPy has flaky support for __qualname__, so some builtin + functions such as float.__new__ will be classified as dynamic. For + this reason only, we created this special routine. Because + builtin-functions are not expected to have closure or globals, + there is no additional hack (compared the one already implemented + in pickle) to protect ourselves from reference cycles. A simple + (reconstructor, newargs, obj.__dict__) tuple is save_reduced. Note + also that PyPy improved their support for __qualname__ in v3.6, so + this routing should be removed when cloudpickle supports only PyPy + 3.6 and later. + """ + rv = (types.FunctionType, (obj.__code__, {}, obj.__name__, + obj.__defaults__, obj.__closure__), + obj.__dict__) + self.save_reduce(*rv, obj=obj) + + dispatch[types.FunctionType] = save_function diff --git a/python/pyspark/cloudpickle/compat.py b/python/pyspark/cloudpickle/compat.py new file mode 100644 index 0000000000000..afa285f62903d --- /dev/null +++ b/python/pyspark/cloudpickle/compat.py @@ -0,0 +1,13 @@ +import sys + + +if sys.version_info < (3, 8): + try: + import pickle5 as pickle # noqa: F401 + from pickle5 import Pickler # noqa: F401 + except ImportError: + import pickle # noqa: F401 + from pickle import _Pickler as Pickler # noqa: F401 +else: + import pickle # noqa: F401 + from _pickle import Pickler # noqa: F401 diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2e105cc38260d..55a5657b64055 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -117,8 +117,10 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ... ValueError:... """ - # In order to prevent SparkContext from being created in executors. - SparkContext._assert_on_driver() + if (conf is None or + conf.get("spark.executor.allowSparkContext", "false").lower() != "true"): + # In order to prevent SparkContext from being created in executors. + SparkContext._assert_on_driver() self._callsite = first_spark_call() or CallSite(None, None, None) if gateway is not None and gateway.gateway_parameters.auth_token is None: @@ -1013,8 +1015,10 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): .. note:: Currently, setting a group ID (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local - properties for each thread on PVM. To work around this, You can use - :meth:`RDD.collectWithJobGroup` for now. + properties for each thread on PVM. + + To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` + environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. """ self._jsc.setJobGroup(groupId, description, interruptOnCancel) @@ -1026,8 +1030,10 @@ def setLocalProperty(self, key, value): .. note:: Currently, setting a local property with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, You can use - :meth:`RDD.collectWithJobGroup`. + for each thread on PVM. + + To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` + environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. """ self._jsc.setLocalProperty(key, value) @@ -1045,8 +1051,10 @@ def setJobDescription(self, value): .. note:: Currently, setting a job description (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate - local properties for each thread on PVM. To work around this, You can use - :meth:`RDD.collectWithJobGroup` for now. + local properties for each thread on PVM. + + To avoid this, enable the pinned thread mode by setting ``PYSPARK_PIN_THREAD`` + environment variable to ``true`` and uses :class:`pyspark.InheritableThread`. """ self._jsc.setJobDescription(value) diff --git a/python/pyspark/find_spark_home.py b/python/pyspark/find_spark_home.py index 920c04009dd11..011c3f5b10a6b 100755 --- a/python/pyspark/find_spark_home.py +++ b/python/pyspark/find_spark_home.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py deleted file mode 100644 index 37a2914ebac05..0000000000000 --- a/python/pyspark/heapq3.py +++ /dev/null @@ -1,890 +0,0 @@ -# -*- encoding: utf-8 -*- -# back ported from CPython 3 -# A. HISTORY OF THE SOFTWARE -# ========================== -# -# Python was created in the early 1990s by Guido van Rossum at Stichting -# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands -# as a successor of a language called ABC. Guido remains Python's -# principal author, although it includes many contributions from others. -# -# In 1995, Guido continued his work on Python at the Corporation for -# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) -# in Reston, Virginia where he released several versions of the -# software. -# -# In May 2000, Guido and the Python core development team moved to -# BeOpen.com to form the BeOpen PythonLabs team. In October of the same -# year, the PythonLabs team moved to Digital Creations (now Zope -# Corporation, see http://www.zope.com). In 2001, the Python Software -# Foundation (PSF, see http://www.python.org/psf/) was formed, a -# non-profit organization created specifically to own Python-related -# Intellectual Property. Zope Corporation is a sponsoring member of -# the PSF. -# -# All Python releases are Open Source (see http://www.opensource.org for -# the Open Source Definition). Historically, most, but not all, Python -# releases have also been GPL-compatible; the table below summarizes -# the various releases. -# -# Release Derived Year Owner GPL- -# from compatible? (1) -# -# 0.9.0 thru 1.2 1991-1995 CWI yes -# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes -# 1.6 1.5.2 2000 CNRI no -# 2.0 1.6 2000 BeOpen.com no -# 1.6.1 1.6 2001 CNRI yes (2) -# 2.1 2.0+1.6.1 2001 PSF no -# 2.0.1 2.0+1.6.1 2001 PSF yes -# 2.1.1 2.1+2.0.1 2001 PSF yes -# 2.2 2.1.1 2001 PSF yes -# 2.1.2 2.1.1 2002 PSF yes -# 2.1.3 2.1.2 2002 PSF yes -# 2.2.1 2.2 2002 PSF yes -# 2.2.2 2.2.1 2002 PSF yes -# 2.2.3 2.2.2 2003 PSF yes -# 2.3 2.2.2 2002-2003 PSF yes -# 2.3.1 2.3 2002-2003 PSF yes -# 2.3.2 2.3.1 2002-2003 PSF yes -# 2.3.3 2.3.2 2002-2003 PSF yes -# 2.3.4 2.3.3 2004 PSF yes -# 2.3.5 2.3.4 2005 PSF yes -# 2.4 2.3 2004 PSF yes -# 2.4.1 2.4 2005 PSF yes -# 2.4.2 2.4.1 2005 PSF yes -# 2.4.3 2.4.2 2006 PSF yes -# 2.4.4 2.4.3 2006 PSF yes -# 2.5 2.4 2006 PSF yes -# 2.5.1 2.5 2007 PSF yes -# 2.5.2 2.5.1 2008 PSF yes -# 2.5.3 2.5.2 2008 PSF yes -# 2.6 2.5 2008 PSF yes -# 2.6.1 2.6 2008 PSF yes -# 2.6.2 2.6.1 2009 PSF yes -# 2.6.3 2.6.2 2009 PSF yes -# 2.6.4 2.6.3 2009 PSF yes -# 2.6.5 2.6.4 2010 PSF yes -# 2.7 2.6 2010 PSF yes -# -# Footnotes: -# -# (1) GPL-compatible doesn't mean that we're distributing Python under -# the GPL. All Python licenses, unlike the GPL, let you distribute -# a modified version without making your changes open source. The -# GPL-compatible licenses make it possible to combine Python with -# other software that is released under the GPL; the others don't. -# -# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, -# because its license has a choice of law clause. According to -# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 -# is "not incompatible" with the GPL. -# -# Thanks to the many outside volunteers who have worked under Guido's -# direction to make these releases possible. -# -# -# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON -# =============================================================== -# -# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 -# -------------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Python Software Foundation -# ("PSF"), and the Individual or Organization ("Licensee") accessing and -# otherwise using this software ("Python") in source or binary form and -# its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, PSF hereby -# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, -# analyze, test, perform and/or display publicly, prepare derivative works, -# distribute, and otherwise use Python alone or in any derivative version, -# provided, however, that PSF's License Agreement and PSF's notice of copyright, -# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, -# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained -# in Python alone or in any derivative version prepared by Licensee. -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python. -# -# 4. PSF is making Python available to Licensee on an "AS IS" -# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. Nothing in this License Agreement shall be deemed to create any -# relationship of agency, partnership, or joint venture between PSF and -# Licensee. This License Agreement does not grant permission to use PSF -# trademarks or trade name in a trademark sense to endorse or promote -# products or services of Licensee, or any third party. -# -# 8. By copying, installing or otherwise using Python, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 -# ------------------------------------------- -# -# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 -# -# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an -# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the -# Individual or Organization ("Licensee") accessing and otherwise using -# this software in source or binary form and its associated -# documentation ("the Software"). -# -# 2. Subject to the terms and conditions of this BeOpen Python License -# Agreement, BeOpen hereby grants Licensee a non-exclusive, -# royalty-free, world-wide license to reproduce, analyze, test, perform -# and/or display publicly, prepare derivative works, distribute, and -# otherwise use the Software alone or in any derivative version, -# provided, however, that the BeOpen Python License is retained in the -# Software, alone or in any derivative version prepared by Licensee. -# -# 3. BeOpen is making the Software available to Licensee on an "AS IS" -# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE -# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS -# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY -# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 5. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 6. This License Agreement shall be governed by and interpreted in all -# respects by the law of the State of California, excluding conflict of -# law provisions. Nothing in this License Agreement shall be deemed to -# create any relationship of agency, partnership, or joint venture -# between BeOpen and Licensee. This License Agreement does not grant -# permission to use BeOpen trademarks or trade names in a trademark -# sense to endorse or promote products or services of Licensee, or any -# third party. As an exception, the "BeOpen Python" logos available at -# http://www.pythonlabs.com/logos.html may be used according to the -# permissions granted on that web page. -# -# 7. By copying, installing or otherwise using the software, Licensee -# agrees to be bound by the terms and conditions of this License -# Agreement. -# -# -# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 -# --------------------------------------- -# -# 1. This LICENSE AGREEMENT is between the Corporation for National -# Research Initiatives, having an office at 1895 Preston White Drive, -# Reston, VA 20191 ("CNRI"), and the Individual or Organization -# ("Licensee") accessing and otherwise using Python 1.6.1 software in -# source or binary form and its associated documentation. -# -# 2. Subject to the terms and conditions of this License Agreement, CNRI -# hereby grants Licensee a nonexclusive, royalty-free, world-wide -# license to reproduce, analyze, test, perform and/or display publicly, -# prepare derivative works, distribute, and otherwise use Python 1.6.1 -# alone or in any derivative version, provided, however, that CNRI's -# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) -# 1995-2001 Corporation for National Research Initiatives; All Rights -# Reserved" are retained in Python 1.6.1 alone or in any derivative -# version prepared by Licensee. Alternately, in lieu of CNRI's License -# Agreement, Licensee may substitute the following text (omitting the -# quotes): "Python 1.6.1 is made available subject to the terms and -# conditions in CNRI's License Agreement. This Agreement together with -# Python 1.6.1 may be located on the Internet using the following -# unique, persistent identifier (known as a handle): 1895.22/1013. This -# Agreement may also be obtained from a proxy server on the Internet -# using the following URL: http://hdl.handle.net/1895.22/1013". -# -# 3. In the event Licensee prepares a derivative work that is based on -# or incorporates Python 1.6.1 or any part thereof, and wants to make -# the derivative work available to others as provided herein, then -# Licensee hereby agrees to include in any such work a brief summary of -# the changes made to Python 1.6.1. -# -# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" -# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND -# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT -# INFRINGE ANY THIRD PARTY RIGHTS. -# -# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, -# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. -# -# 6. This License Agreement will automatically terminate upon a material -# breach of its terms and conditions. -# -# 7. This License Agreement shall be governed by the federal -# intellectual property law of the United States, including without -# limitation the federal copyright law, and, to the extent such -# U.S. federal law does not apply, by the law of the Commonwealth of -# Virginia, excluding Virginia's conflict of law provisions. -# Notwithstanding the foregoing, with regard to derivative works based -# on Python 1.6.1 that incorporate non-separable material that was -# previously distributed under the GNU General Public License (GPL), the -# law of the Commonwealth of Virginia shall govern this License -# Agreement only as to issues arising under or with respect to -# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this -# License Agreement shall be deemed to create any relationship of -# agency, partnership, or joint venture between CNRI and Licensee. This -# License Agreement does not grant permission to use CNRI trademarks or -# trade name in a trademark sense to endorse or promote products or -# services of Licensee, or any third party. -# -# 8. By clicking on the "ACCEPT" button where indicated, or by copying, -# installing or otherwise using Python 1.6.1, Licensee agrees to be -# bound by the terms and conditions of this License Agreement. -# -# ACCEPT -# -# -# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 -# -------------------------------------------------- -# -# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, -# The Netherlands. All rights reserved. -# -# Permission to use, copy, modify, and distribute this software and its -# documentation for any purpose and without fee is hereby granted, -# provided that the above copyright notice appear in all copies and that -# both that copyright notice and this permission notice appear in -# supporting documentation, and that the name of Stichting Mathematisch -# Centrum or CWI not be used in advertising or publicity pertaining to -# distribution of the software without specific, written prior -# permission. -# -# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO -# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND -# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE -# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES -# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN -# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT -# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. -"""Heap queue algorithm (a.k.a. priority queue). - -Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for -all k, counting elements from 0. For the sake of comparison, -non-existing elements are considered to be infinite. The interesting -property of a heap is that a[0] is always its smallest element. - -Usage: - -heap = [] # creates an empty heap -heappush(heap, item) # pushes a new item on the heap -item = heappop(heap) # pops the smallest item from the heap -item = heap[0] # smallest item on the heap without popping it -heapify(x) # transforms list into a heap, in-place, in linear time -item = heapreplace(heap, item) # pops and returns smallest item, and adds - # new item; the heap size is unchanged - -Our API differs from textbook heap algorithms as follows: - -- We use 0-based indexing. This makes the relationship between the - index for a node and the indexes for its children slightly less - obvious, but is more suitable since Python uses 0-based indexing. - -- Our heappop() method returns the smallest item, not the largest. - -These two make it possible to view the heap as a regular Python list -without surprises: heap[0] is the smallest item, and heap.sort() -maintains the heap invariant! -""" - -# Original code by Kevin O'Connor, augmented by Tim Peters and Raymond Hettinger - -__about__ = """Heap queues - -[explanation by François Pinard] - -Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for -all k, counting elements from 0. For the sake of comparison, -non-existing elements are considered to be infinite. The interesting -property of a heap is that a[0] is always its smallest element. - -The strange invariant above is meant to be an efficient memory -representation for a tournament. The numbers below are `k', not a[k]: - - 0 - - 1 2 - - 3 4 5 6 - - 7 8 9 10 11 12 13 14 - - 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 - - -In the tree above, each cell `k' is topping `2*k+1' and `2*k+2'. In -an usual binary tournament we see in sports, each cell is the winner -over the two cells it tops, and we can trace the winner down the tree -to see all opponents s/he had. However, in many computer applications -of such tournaments, we do not need to trace the history of a winner. -To be more memory efficient, when a winner is promoted, we try to -replace it by something else at a lower level, and the rule becomes -that a cell and the two cells it tops contain three different items, -but the top cell "wins" over the two topped cells. - -If this heap invariant is protected at all time, index 0 is clearly -the overall winner. The simplest algorithmic way to remove it and -find the "next" winner is to move some loser (let's say cell 30 in the -diagram above) into the 0 position, and then percolate this new 0 down -the tree, exchanging values, until the invariant is re-established. -This is clearly logarithmic on the total number of items in the tree. -By iterating over all items, you get an O(n ln n) sort. - -A nice feature of this sort is that you can efficiently insert new -items while the sort is going on, provided that the inserted items are -not "better" than the last 0'th element you extracted. This is -especially useful in simulation contexts, where the tree holds all -incoming events, and the "win" condition means the smallest scheduled -time. When an event schedule other events for execution, they are -scheduled into the future, so they can easily go into the heap. So, a -heap is a good structure for implementing schedulers (this is what I -used for my MIDI sequencer :-). - -Various structures for implementing schedulers have been extensively -studied, and heaps are good for this, as they are reasonably speedy, -the speed is almost constant, and the worst case is not much different -than the average case. However, there are other representations which -are more efficient overall, yet the worst cases might be terrible. - -Heaps are also very useful in big disk sorts. You most probably all -know that a big sort implies producing "runs" (which are pre-sorted -sequences, which size is usually related to the amount of CPU memory), -followed by a merging passes for these runs, which merging is often -very cleverly organised[1]. It is very important that the initial -sort produces the longest runs possible. Tournaments are a good way -to that. If, using all the memory available to hold a tournament, you -replace and percolate items that happen to fit the current run, you'll -produce runs which are twice the size of the memory for random input, -and much better for input fuzzily ordered. - -Moreover, if you output the 0'th item on disk and get an input which -may not fit in the current tournament (because the value "wins" over -the last output value), it cannot fit in the heap, so the size of the -heap decreases. The freed memory could be cleverly reused immediately -for progressively building a second heap, which grows at exactly the -same rate the first heap is melting. When the first heap completely -vanishes, you switch heaps and start a new run. Clever and quite -effective! - -In a word, heaps are useful memory structures to know. I use them in -a few applications, and I think it is good to keep a `heap' module -around. :-) - --------------------- -[1] The disk balancing algorithms which are current, nowadays, are -more annoying than clever, and this is a consequence of the seeking -capabilities of the disks. On devices which cannot seek, like big -tape drives, the story was quite different, and one had to be very -clever to ensure (far in advance) that each tape movement will be the -most effective possible (that is, will best participate at -"progressing" the merge). Some tapes were even able to read -backwards, and this was also used to avoid the rewinding time. -Believe me, real good tape sorts were quite spectacular to watch! -From all times, sorting has always been a Great Art! :-) -""" - -__all__ = ['heappush', 'heappop', 'heapify', 'heapreplace', 'merge', - 'nlargest', 'nsmallest', 'heappushpop'] - -def heappush(heap, item): - """Push item onto heap, maintaining the heap invariant.""" - heap.append(item) - _siftdown(heap, 0, len(heap)-1) - -def heappop(heap): - """Pop the smallest item off the heap, maintaining the heap invariant.""" - lastelt = heap.pop() # raises appropriate IndexError if heap is empty - if heap: - returnitem = heap[0] - heap[0] = lastelt - _siftup(heap, 0) - return returnitem - return lastelt - -def heapreplace(heap, item): - """Pop and return the current smallest value, and add the new item. - - This is more efficient than heappop() followed by heappush(), and can be - more appropriate when using a fixed-size heap. Note that the value - returned may be larger than item! That constrains reasonable uses of - this routine unless written as part of a conditional replacement: - - if item > heap[0]: - item = heapreplace(heap, item) - """ - returnitem = heap[0] # raises appropriate IndexError if heap is empty - heap[0] = item - _siftup(heap, 0) - return returnitem - -def heappushpop(heap, item): - """Fast version of a heappush followed by a heappop.""" - if heap and heap[0] < item: - item, heap[0] = heap[0], item - _siftup(heap, 0) - return item - -def heapify(x): - """Transform list into a heap, in-place, in O(len(x)) time.""" - n = len(x) - # Transform bottom-up. The largest index there's any point to looking at - # is the largest with a child index in-range, so must have 2*i + 1 < n, - # or i < (n-1)/2. If n is even = 2*j, this is (2*j-1)/2 = j-1/2 so - # j-1 is the largest, which is n//2 - 1. If n is odd = 2*j+1, this is - # (2*j+1-1)/2 = j so j-1 is the largest, and that's again n//2-1. - for i in reversed(range(n//2)): - _siftup(x, i) - -def _heappop_max(heap): - """Maxheap version of a heappop.""" - lastelt = heap.pop() # raises appropriate IndexError if heap is empty - if heap: - returnitem = heap[0] - heap[0] = lastelt - _siftup_max(heap, 0) - return returnitem - return lastelt - -def _heapreplace_max(heap, item): - """Maxheap version of a heappop followed by a heappush.""" - returnitem = heap[0] # raises appropriate IndexError if heap is empty - heap[0] = item - _siftup_max(heap, 0) - return returnitem - -def _heapify_max(x): - """Transform list into a maxheap, in-place, in O(len(x)) time.""" - n = len(x) - for i in reversed(range(n//2)): - _siftup_max(x, i) - -# 'heap' is a heap at all indices >= startpos, except possibly for pos. pos -# is the index of a leaf with a possibly out-of-order value. Restore the -# heap invariant. -def _siftdown(heap, startpos, pos): - newitem = heap[pos] - # Follow the path to the root, moving parents down until finding a place - # newitem fits. - while pos > startpos: - parentpos = (pos - 1) >> 1 - parent = heap[parentpos] - if newitem < parent: - heap[pos] = parent - pos = parentpos - continue - break - heap[pos] = newitem - -# The child indices of heap index pos are already heaps, and we want to make -# a heap at index pos too. We do this by bubbling the smaller child of -# pos up (and so on with that child's children, etc) until hitting a leaf, -# then using _siftdown to move the oddball originally at index pos into place. -# -# We *could* break out of the loop as soon as we find a pos where newitem <= -# both its children, but turns out that's not a good idea, and despite that -# many books write the algorithm that way. During a heap pop, the last array -# element is sifted in, and that tends to be large, so that comparing it -# against values starting from the root usually doesn't pay (= usually doesn't -# get us out of the loop early). See Knuth, Volume 3, where this is -# explained and quantified in an exercise. -# -# Cutting the # of comparisons is important, since these routines have no -# way to extract "the priority" from an array element, so that intelligence -# is likely to be hiding in custom comparison methods, or in array elements -# storing (priority, record) tuples. Comparisons are thus potentially -# expensive. -# -# On random arrays of length 1000, making this change cut the number of -# comparisons made by heapify() a little, and those made by exhaustive -# heappop() a lot, in accord with theory. Here are typical results from 3 -# runs (3 just to demonstrate how small the variance is): -# -# Compares needed by heapify Compares needed by 1000 heappops -# -------------------------- -------------------------------- -# 1837 cut to 1663 14996 cut to 8680 -# 1855 cut to 1659 14966 cut to 8678 -# 1847 cut to 1660 15024 cut to 8703 -# -# Building the heap by using heappush() 1000 times instead required -# 2198, 2148, and 2219 compares: heapify() is more efficient, when -# you can use it. -# -# The total compares needed by list.sort() on the same lists were 8627, -# 8627, and 8632 (this should be compared to the sum of heapify() and -# heappop() compares): list.sort() is (unsurprisingly!) more efficient -# for sorting. - -def _siftup(heap, pos): - endpos = len(heap) - startpos = pos - newitem = heap[pos] - # Bubble up the smaller child until hitting a leaf. - childpos = 2*pos + 1 # leftmost child position - while childpos < endpos: - # Set childpos to index of smaller child. - rightpos = childpos + 1 - if rightpos < endpos and not heap[childpos] < heap[rightpos]: - childpos = rightpos - # Move the smaller child up. - heap[pos] = heap[childpos] - pos = childpos - childpos = 2*pos + 1 - # The leaf at pos is empty now. Put newitem there, and bubble it up - # to its final resting place (by sifting its parents down). - heap[pos] = newitem - _siftdown(heap, startpos, pos) - -def _siftdown_max(heap, startpos, pos): - 'Maxheap variant of _siftdown' - newitem = heap[pos] - # Follow the path to the root, moving parents down until finding a place - # newitem fits. - while pos > startpos: - parentpos = (pos - 1) >> 1 - parent = heap[parentpos] - if parent < newitem: - heap[pos] = parent - pos = parentpos - continue - break - heap[pos] = newitem - -def _siftup_max(heap, pos): - 'Maxheap variant of _siftup' - endpos = len(heap) - startpos = pos - newitem = heap[pos] - # Bubble up the larger child until hitting a leaf. - childpos = 2*pos + 1 # leftmost child position - while childpos < endpos: - # Set childpos to index of larger child. - rightpos = childpos + 1 - if rightpos < endpos and not heap[rightpos] < heap[childpos]: - childpos = rightpos - # Move the larger child up. - heap[pos] = heap[childpos] - pos = childpos - childpos = 2*pos + 1 - # The leaf at pos is empty now. Put newitem there, and bubble it up - # to its final resting place (by sifting its parents down). - heap[pos] = newitem - _siftdown_max(heap, startpos, pos) - -def merge(iterables, key=None, reverse=False): - '''Merge multiple sorted inputs into a single sorted output. - - Similar to sorted(itertools.chain(*iterables)) but returns a generator, - does not pull the data into memory all at once, and assumes that each of - the input streams is already sorted (smallest to largest). - - >>> list(merge([1,3,5,7], [0,2,4,8], [5,10,15,20], [], [25])) - [0, 1, 2, 3, 4, 5, 5, 7, 8, 10, 15, 20, 25] - - If *key* is not None, applies a key function to each element to determine - its sort order. - - >>> list(merge(['dog', 'horse'], ['cat', 'fish', 'kangaroo'], key=len)) - ['dog', 'cat', 'fish', 'horse', 'kangaroo'] - - ''' - - h = [] - h_append = h.append - - if reverse: - _heapify = _heapify_max - _heappop = _heappop_max - _heapreplace = _heapreplace_max - direction = -1 - else: - _heapify = heapify - _heappop = heappop - _heapreplace = heapreplace - direction = 1 - - if key is None: - for order, it in enumerate(map(iter, iterables)): - try: - h_append([next(it), order * direction, it]) - except StopIteration: - pass - _heapify(h) - while len(h) > 1: - try: - while True: - value, order, it = s = h[0] - yield value - s[0] = next(it) # raises StopIteration when exhausted - _heapreplace(h, s) # restore heap condition - except StopIteration: - _heappop(h) # remove empty iterator - if h: - # fast case when only a single iterator remains - value, order, it = h[0] - yield value - for value in it: - yield value - return - - for order, it in enumerate(map(iter, iterables)): - try: - value = next(it) - h_append([key(value), order * direction, value, it]) - except StopIteration: - pass - _heapify(h) - while len(h) > 1: - try: - while True: - key_value, order, value, it = s = h[0] - yield value - value = next(it) - s[0] = key(value) - s[2] = value - _heapreplace(h, s) - except StopIteration: - _heappop(h) - if h: - key_value, order, value, it = h[0] - yield value - for value in it: - yield value - - -# Algorithm notes for nlargest() and nsmallest() -# ============================================== -# -# Make a single pass over the data while keeping the k most extreme values -# in a heap. Memory consumption is limited to keeping k values in a list. -# -# Measured performance for random inputs: -# -# number of comparisons -# n inputs k-extreme values (average of 5 trials) % more than min() -# ------------- ---------------- --------------------- ----------------- -# 1,000 100 3,317 231.7% -# 10,000 100 14,046 40.5% -# 100,000 100 105,749 5.7% -# 1,000,000 100 1,007,751 0.8% -# 10,000,000 100 10,009,401 0.1% -# -# Theoretical number of comparisons for k smallest of n random inputs: -# -# Step Comparisons Action -# ---- -------------------------- --------------------------- -# 1 1.66 * k heapify the first k-inputs -# 2 n - k compare remaining elements to top of heap -# 3 k * (1 + lg2(k)) * ln(n/k) replace the topmost value on the heap -# 4 k * lg2(k) - (k/2) final sort of the k most extreme values -# -# Combining and simplifying for a rough estimate gives: -# -# comparisons = n + k * (log(k, 2) * log(n/k) + log(k, 2) + log(n/k)) -# -# Computing the number of comparisons for step 3: -# ----------------------------------------------- -# * For the i-th new value from the iterable, the probability of being in the -# k most extreme values is k/i. For example, the probability of the 101st -# value seen being in the 100 most extreme values is 100/101. -# * If the value is a new extreme value, the cost of inserting it into the -# heap is 1 + log(k, 2). -# * The probability times the cost gives: -# (k/i) * (1 + log(k, 2)) -# * Summing across the remaining n-k elements gives: -# sum((k/i) * (1 + log(k, 2)) for i in range(k+1, n+1)) -# * This reduces to: -# (H(n) - H(k)) * k * (1 + log(k, 2)) -# * Where H(n) is the n-th harmonic number estimated by: -# gamma = 0.5772156649 -# H(n) = log(n, e) + gamma + 1 / (2 * n) -# http://en.wikipedia.org/wiki/Harmonic_series_(mathematics)#Rate_of_divergence -# * Substituting the H(n) formula: -# comparisons = k * (1 + log(k, 2)) * (log(n/k, e) + (1/n - 1/k) / 2) -# -# Worst-case for step 3: -# ---------------------- -# In the worst case, the input data is reversed sorted so that every new element -# must be inserted in the heap: -# -# comparisons = 1.66 * k + log(k, 2) * (n - k) -# -# Alternative Algorithms -# ---------------------- -# Other algorithms were not used because they: -# 1) Took much more auxiliary memory, -# 2) Made multiple passes over the data. -# 3) Made more comparisons in common cases (small k, large n, semi-random input). -# See the more detailed comparison of approach at: -# http://code.activestate.com/recipes/577573-compare-algorithms-for-heapqsmallest - -def nsmallest(n, iterable, key=None): - """Find the n smallest elements in a dataset. - - Equivalent to: sorted(iterable, key=key)[:n] - """ - - # Short-cut for n==1 is to use min() - if n == 1: - it = iter(iterable) - sentinel = object() - if key is None: - result = min(it, default=sentinel) - else: - result = min(it, default=sentinel, key=key) - return [] if result is sentinel else [result] - - # When n>=size, it's faster to use sorted() - try: - size = len(iterable) - except (TypeError, AttributeError): - pass - else: - if n >= size: - return sorted(iterable, key=key)[:n] - - # When key is none, use simpler decoration - if key is None: - it = iter(iterable) - # put the range(n) first so that zip() doesn't - # consume one too many elements from the iterator - result = [(elem, i) for i, elem in zip(range(n), it)] - if not result: - return result - _heapify_max(result) - top = result[0][0] - order = n - _heapreplace = _heapreplace_max - for elem in it: - if elem < top: - _heapreplace(result, (elem, order)) - top = result[0][0] - order += 1 - result.sort() - return [r[0] for r in result] - - # General case, slowest method - it = iter(iterable) - result = [(key(elem), i, elem) for i, elem in zip(range(n), it)] - if not result: - return result - _heapify_max(result) - top = result[0][0] - order = n - _heapreplace = _heapreplace_max - for elem in it: - k = key(elem) - if k < top: - _heapreplace(result, (k, order, elem)) - top = result[0][0] - order += 1 - result.sort() - return [r[2] for r in result] - -def nlargest(n, iterable, key=None): - """Find the n largest elements in a dataset. - - Equivalent to: sorted(iterable, key=key, reverse=True)[:n] - """ - - # Short-cut for n==1 is to use max() - if n == 1: - it = iter(iterable) - sentinel = object() - if key is None: - result = max(it, default=sentinel) - else: - result = max(it, default=sentinel, key=key) - return [] if result is sentinel else [result] - - # When n>=size, it's faster to use sorted() - try: - size = len(iterable) - except (TypeError, AttributeError): - pass - else: - if n >= size: - return sorted(iterable, key=key, reverse=True)[:n] - - # When key is none, use simpler decoration - if key is None: - it = iter(iterable) - result = [(elem, i) for i, elem in zip(range(0, -n, -1), it)] - if not result: - return result - heapify(result) - top = result[0][0] - order = -n - _heapreplace = heapreplace - for elem in it: - if top < elem: - _heapreplace(result, (elem, order)) - top = result[0][0] - order -= 1 - result.sort(reverse=True) - return [r[0] for r in result] - - # General case, slowest method - it = iter(iterable) - result = [(key(elem), i, elem) for i, elem in zip(range(0, -n, -1), it)] - if not result: - return result - heapify(result) - top = result[0][0] - order = -n - _heapreplace = heapreplace - for elem in it: - k = key(elem) - if top < k: - _heapreplace(result, (k, order, elem)) - top = result[0][0] - order -= 1 - result.sort(reverse=True) - return [r[2] for r in result] - -# If available, use C implementation -try: - from _heapq import * -except ImportError: - pass -try: - from _heapq import _heapreplace_max -except ImportError: - pass -try: - from _heapq import _heapify_max -except ImportError: - pass -try: - from _heapq import _heappop_max -except ImportError: - pass - - -if __name__ == "__main__": - import doctest - import sys - (failure_count, test_count) = doctest.testmod() - if failure_count: - sys.exit(-1) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4f2d33adbc7e7..a223a9be4d8ea 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -51,6 +51,8 @@ 'BinaryRandomForestClassificationTrainingSummary', 'NaiveBayes', 'NaiveBayesModel', 'MultilayerPerceptronClassifier', 'MultilayerPerceptronClassificationModel', + 'MultilayerPerceptronClassificationSummary', + 'MultilayerPerceptronClassificationTrainingSummary', 'OneVsRest', 'OneVsRestModel', 'FMClassifier', 'FMClassificationModel', 'FMClassificationSummary', 'FMClassificationTrainingSummary'] @@ -513,6 +515,12 @@ class _LinearSVCParams(_ClassifierParams, HasRegParam, HasMaxIter, HasFitInterce " all predictions 0.0 and -Inf will make all predictions 1.0.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_LinearSVCParams, self).__init__(*args) + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, + standardization=True, threshold=0.0, aggregationDepth=2, + blockSize=1) + @inherit_doc class LinearSVC(_JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable): @@ -581,6 +589,8 @@ class LinearSVC(_JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadabl True >>> model.intercept == model2.intercept True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 2.2.0 """ @@ -599,9 +609,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(LinearSVC, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LinearSVC", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, - standardization=True, threshold=0.0, aggregationDepth=2, - blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -817,6 +824,11 @@ class _LogisticRegressionParams(_ProbabilisticClassifierParams, HasRegParam, "classes for multinomial regression.", typeConverter=TypeConverters.toVector) + def __init__(self, *args): + super(_LogisticRegressionParams, self).__init__(*args) + self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", + blockSize=1) + @since("1.4.0") def setThreshold(self, value): """ @@ -1010,6 +1022,8 @@ class LogisticRegression(_JavaProbabilisticClassifier, _LogisticRegressionParams True >>> model2 LogisticRegressionModel: uid=..., numClasses=2, numFeatures=2 + >>> blorModel.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.3.0 """ @@ -1038,8 +1052,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(LogisticRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LogisticRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", - blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) self._checkThresholdConsistency() @@ -1306,7 +1318,12 @@ class _DecisionTreeClassifierParams(_DecisionTreeParams, _TreeClassifierParams): """ Params for :py:class:`DecisionTreeClassifier` and :py:class:`DecisionTreeClassificationModel`. """ - pass + + def __init__(self, *args): + super(_DecisionTreeClassifierParams, self).__init__(*args) + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="gini", leafCol="", minWeightFractionPerNode=0.0) @inherit_doc @@ -1373,7 +1390,8 @@ class DecisionTreeClassifier(_JavaProbabilisticClassifier, _DecisionTreeClassifi >>> model2 = DecisionTreeClassificationModel.load(model_path) >>> model.featureImportances == model2.featureImportances True - + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> df3 = spark.createDataFrame([ ... (1.0, 0.2, Vectors.dense(1.0)), ... (1.0, 0.8, Vectors.dense(1.0)), @@ -1405,9 +1423,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(DecisionTreeClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.DecisionTreeClassifier", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="gini", leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1541,7 +1556,14 @@ class _RandomForestClassifierParams(_RandomForestParams, _TreeClassifierParams): """ Params for :py:class:`RandomForestClassifier` and :py:class:`RandomForestClassificationModel`. """ - pass + + def __init__(self, *args): + super(_RandomForestClassifierParams, self).__init__(*args) + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="gini", numTrees=20, featureSubsetStrategy="auto", + subsamplingRate=1.0, leafCol="", minWeightFractionPerNode=0.0, + bootstrap=True) @inherit_doc @@ -1613,6 +1635,8 @@ class RandomForestClassifier(_JavaProbabilisticClassifier, _RandomForestClassifi >>> model2 = RandomForestClassificationModel.load(model_path) >>> model.featureImportances == model2.featureImportances True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.4.0 """ @@ -1635,11 +1659,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(RandomForestClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.RandomForestClassifier", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="gini", numTrees=20, featureSubsetStrategy="auto", - subsamplingRate=1.0, leafCol="", minWeightFractionPerNode=0.0, - bootstrap=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1883,6 +1902,14 @@ class _GBTClassifierParams(_GBTParams, _HasVarianceImpurity): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_GBTClassifierParams, self).__init__(*args) + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + lossType="logistic", maxIter=20, stepSize=0.1, subsamplingRate=1.0, + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, + leafCol="", minWeightFractionPerNode=0.0) + @since("1.4.0") def getLossType(self): """ @@ -1974,6 +2001,8 @@ class GBTClassifier(_JavaProbabilisticClassifier, _GBTClassifierParams, True >>> model.treeWeights == model2.treeWeights True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.trees [DecisionTreeRegressionModel...depth=..., DecisionTreeRegressionModel...] >>> validation = spark.createDataFrame([(0.0, Vectors.dense(-1.0),)], @@ -2010,11 +2039,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(GBTClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.GBTClassifier", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - lossType="logistic", maxIter=20, stepSize=0.1, subsamplingRate=1.0, - impurity="variance", featureSubsetStrategy="all", validationTol=0.01, - leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2212,6 +2236,10 @@ class _NaiveBayesParams(_PredictorParams, HasWeightCol): "and gaussian.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_NaiveBayesParams, self).__init__(*args) + self._setDefault(smoothing=1.0, modelType="multinomial") + @since("1.5.0") def getSmoothing(self): """ @@ -2295,6 +2323,8 @@ class NaiveBayes(_JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, True >>> model.theta == model2.theta True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> nb = nb.setThresholds([0.01, 10.00]) >>> model3 = nb.fit(df) >>> result = model3.transform(test0).head() @@ -2330,7 +2360,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(NaiveBayes, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.NaiveBayes", self.uid) - self._setDefault(smoothing=1.0, modelType="multinomial") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2422,8 +2451,8 @@ class _MultilayerPerceptronParams(_ProbabilisticClassifierParams, HasSeed, HasMa initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", typeConverter=TypeConverters.toVector) - def __init__(self): - super(_MultilayerPerceptronParams, self).__init__() + def __init__(self, *args): + super(_MultilayerPerceptronParams, self).__init__(*args) self._setDefault(maxIter=100, tol=1E-6, blockSize=128, stepSize=0.03, solver="l-bfgs") @since("1.6.0") @@ -2505,6 +2534,8 @@ class MultilayerPerceptronClassifier(_JavaProbabilisticClassifier, _MultilayerPe True >>> model.weights == model2.weights True + >>> model.transform(testDF).take(1) == model2.transform(testDF).take(1) + True >>> mlp2 = mlp2.setInitialWeights(list(range(0, 12))) >>> model3 = mlp2.fit(df) >>> model3.weights != model2.weights @@ -2606,7 +2637,7 @@ def setSolver(self, value): class MultilayerPerceptronClassificationModel(_JavaProbabilisticClassificationModel, _MultilayerPerceptronParams, JavaMLWritable, - JavaMLReadable): + JavaMLReadable, HasTrainingSummary): """ Model fitted by MultilayerPerceptronClassifier. @@ -2621,6 +2652,51 @@ def weights(self): """ return self._call_java("weights") + @since("3.1.0") + def summary(self): + """ + Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + trained on the training set. An exception is thrown if `trainingSummary is None`. + """ + if self.hasSummary: + return MultilayerPerceptronClassificationTrainingSummary( + super(MultilayerPerceptronClassificationModel, self).summary) + else: + raise RuntimeError("No training summary available for this %s" % + self.__class__.__name__) + + @since("3.1.0") + def evaluate(self, dataset): + """ + Evaluates the model on a test dataset. + + :param dataset: + Test dataset to evaluate model on, where dataset is an + instance of :py:class:`pyspark.sql.DataFrame` + """ + if not isinstance(dataset, DataFrame): + raise ValueError("dataset must be a DataFrame but got %s." % type(dataset)) + java_mlp_summary = self._call_java("evaluate", dataset) + return MultilayerPerceptronClassificationSummary(java_mlp_summary) + + +class MultilayerPerceptronClassificationSummary(_ClassificationSummary): + """ + Abstraction for MultilayerPerceptronClassifier Results for a given model. + .. versionadded:: 3.1.0 + """ + pass + + +@inherit_doc +class MultilayerPerceptronClassificationTrainingSummary(MultilayerPerceptronClassificationSummary, + _TrainingSummary): + """ + Abstraction for MultilayerPerceptronClassifier Training results. + .. versionadded:: 3.1.0 + """ + pass + class _OneVsRestParams(_ClassifierParams, HasWeightCol): """ @@ -2679,6 +2755,8 @@ class OneVsRest(Estimator, _OneVsRestParams, HasParallelism, JavaMLReadable, Jav >>> model2 = OneVsRestModel.load(model_path) >>> model2.transform(test0).head().newPrediction 0.0 + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.transform(test2).columns ['features', 'rawPrediction', 'newPrediction'] @@ -3104,6 +3182,17 @@ class FMClassifier(_JavaProbabilisticClassifier, _FactorizationMachinesParams, J DenseVector([14.8232]) >>> model.factors DenseMatrix(1, 2, [0.0163, -0.0051], 1) + >>> model_path = temp_path + "/fm_model" + >>> model.save(model_path) + >>> model2 = FMClassificationModel.load(model_path) + >>> model2.intercept + -7.316665276826291 + >>> model2.linear + DenseVector([14.8232]) + >>> model2.factors + DenseMatrix(1, 2, [0.0163, -0.0051], 1) + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 3.0.0 """ diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 54a184bc081ee..2d70f876849f8 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -109,6 +109,10 @@ class _GaussianMixtureParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC k = Param(Params._dummy(), "k", "Number of independent Gaussians in the mixture model. " + "Must be > 1.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_GaussianMixtureParams, self).__init__(*args) + self._setDefault(k=2, tol=0.01, maxIter=100, aggregationDepth=2, blockSize=1) + @since("2.0.0") def getK(self): """ @@ -321,6 +325,8 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav Row(mean=DenseVector([0.825, 0.8675])) >>> model2.gaussiansDF.select("cov").head() Row(cov=DenseMatrix(2, 2, [0.0056, -0.0051, -0.0051, 0.0046], False)) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True >>> gm2.setWeightCol("weight") GaussianMixture... @@ -339,7 +345,6 @@ def __init__(self, featuresCol="features", predictionCol="prediction", k=2, super(GaussianMixture, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.GaussianMixture", self.uid) - self._setDefault(k=2, tol=0.01, maxIter=100, aggregationDepth=2, blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -500,6 +505,11 @@ class _KMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, HasTo initSteps = Param(Params._dummy(), "initSteps", "The number of steps for k-means|| " + "initialization mode. Must be > 0.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_KMeansParams, self).__init__(*args) + self._setDefault(k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, + distanceMeasure="euclidean") + @since("1.5.0") def getK(self): """ @@ -629,6 +639,8 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): array([ True, True], dtype=bool) >>> model.clusterCenters()[1] == model2.clusterCenters()[1] array([ True, True], dtype=bool) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -644,8 +656,6 @@ def __init__(self, featuresCol="features", predictionCol="prediction", k=2, """ super(KMeans, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.KMeans", self.uid) - self._setDefault(k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, - distanceMeasure="euclidean") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -754,6 +764,10 @@ class _BisectingKMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionC "proportion of points (if < 1.0) of a divisible cluster.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_BisectingKMeansParams, self).__init__(*args) + self._setDefault(maxIter=20, k=4, minDivisibleClusterSize=1.0) + @since("2.0.0") def getK(self): """ @@ -904,6 +918,8 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav array([ True, True], dtype=bool) >>> model.clusterCenters()[1] == model2.clusterCenters()[1] array([ True, True], dtype=bool) + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -920,7 +936,6 @@ def __init__(self, featuresCol="features", predictionCol="prediction", maxIter=2 super(BisectingKMeans, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.BisectingKMeans", self.uid) - self._setDefault(maxIter=20, k=4, minDivisibleClusterSize=1.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1063,6 +1078,13 @@ class _LDAParams(HasMaxIter, HasFeaturesCol, HasSeed, HasCheckpointInterval): " partition is lost, so set this bit with care.", TypeConverters.toBoolean) + def __init__(self, *args): + super(_LDAParams, self).__init__(*args) + self._setDefault(maxIter=20, checkpointInterval=10, + k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51, + subsamplingRate=0.05, optimizeDocConcentration=True, + topicDistributionCol="topicDistribution", keepLastCheckpoint=True) + @since("2.0.0") def getK(self): """ @@ -1373,6 +1395,8 @@ class LDA(JavaEstimator, _LDAParams, JavaMLReadable, JavaMLWritable): >>> local_model_path = temp_path + "/lda_local_model" >>> localModel.save(local_model_path) >>> sameLocalModel = LocalLDAModel.load(local_model_path) + >>> model.transform(df).take(1) == sameLocalModel.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1392,10 +1416,6 @@ def __init__(self, featuresCol="features", maxIter=20, seed=None, checkpointInte """ super(LDA, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid) - self._setDefault(maxIter=20, checkpointInterval=10, - k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51, - subsamplingRate=0.05, optimizeDocConcentration=True, - topicDistributionCol="topicDistribution", keepLastCheckpoint=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1588,6 +1608,10 @@ class _PowerIterationClusteringParams(HasMaxIter, HasWeightCol): "Name of the input column for destination vertex IDs.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_PowerIterationClusteringParams, self).__init__(*args) + self._setDefault(k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst") + @since("2.4.0") def getK(self): """ @@ -1661,6 +1685,8 @@ class PowerIterationClustering(_PowerIterationClusteringParams, JavaParams, Java 2 >>> pic2.getMaxIter() 40 + >>> pic2.assignClusters(df).take(6) == assignments.take(6) + True .. versionadded:: 2.4.0 """ @@ -1675,7 +1701,6 @@ def __init__(self, k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst super(PowerIterationClustering, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.clustering.PowerIterationClustering", self.uid) - self._setDefault(k=2, maxIter=20, initMode="random", srcCol="src", dstCol="dst") kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index a319dace6869a..2220293d54ba4 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -92,6 +92,8 @@ class Binarizer(JavaTransformer, HasThreshold, HasThresholds, HasInputCol, HasOu >>> loadedBinarizer = Binarizer.load(binarizerPath) >>> loadedBinarizer.getThreshold() == binarizer.getThreshold() True + >>> loadedBinarizer.transform(df).take(1) == binarizer.transform(df).take(1) + True >>> df2 = spark.createDataFrame([(0.5, 0.3)], ["values1", "values2"]) >>> binarizer2 = Binarizer(thresholds=[0.0, 1.0]) >>> binarizer2.setInputCols(["values1", "values2"]).setOutputCols(["output1", "output2"]) @@ -195,6 +197,10 @@ class _LSHParams(HasInputCol, HasOutputCol): "and decreasing it improves the running performance.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_LSHParams, self).__init__(*args) + self._setDefault(numHashTables=1) + def getNumHashTables(self): """ Gets the value of numHashTables or its default value. @@ -390,7 +396,6 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, super(BucketedRandomProjectionLSH, self).__init__() self._java_obj = \ self._new_java_obj("org.apache.spark.ml.feature.BucketedRandomProjectionLSH", self.uid) - self._setDefault(numHashTables=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -477,6 +482,8 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> loadedBucketizer = Bucketizer.load(bucketizerPath) >>> loadedBucketizer.getSplits() == bucketizer.getSplits() True + >>> loadedBucketizer.transform(df).take(1) == bucketizer.transform(df).take(1) + True >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 @@ -733,6 +740,8 @@ class CountVectorizer(JavaEstimator, _CountVectorizerParams, JavaMLReadable, Jav >>> loadedModel = CountVectorizerModel.load(modelPath) >>> loadedModel.vocabulary == model.vocabulary True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True >>> fromVocabModel = CountVectorizerModel.from_vocabulary(["a", "b", "c"], ... inputCol="raw", outputCol="vectors") >>> fromVocabModel.transform(df).show(truncate=False) @@ -920,6 +929,8 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit >>> dctPath = temp_path + "/dct" >>> dct.save(dctPath) >>> loadedDtc = DCT.load(dctPath) + >>> loadedDtc.transform(df1).take(1) == dct.transform(df1).take(1) + True >>> loadedDtc.getInverse() False @@ -1003,6 +1014,8 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada >>> loadedEp = ElementwiseProduct.load(elementwiseProductPath) >>> loadedEp.getScalingVec() == ep.getScalingVec() True + >>> loadedEp.transform(df).take(1) == ep.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -1201,6 +1214,8 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, Java >>> loadedHashingTF = HashingTF.load(hashingTFPath) >>> loadedHashingTF.getNumFeatures() == hashingTF.getNumFeatures() True + >>> loadedHashingTF.transform(df).take(1) == hashingTF.transform(df).take(1) + True >>> hashingTF.indexOf("b") 5 @@ -1292,6 +1307,10 @@ def getMinDocFreq(self): """ return self.getOrDefault(self.minDocFreq) + def __init__(self, *args): + super(_IDFParams, self).__init__(*args) + self._setDefault(minDocFreq=0) + @inherit_doc class IDF(JavaEstimator, _IDFParams, JavaMLReadable, JavaMLWritable): @@ -1345,7 +1364,6 @@ def __init__(self, minDocFreq=0, inputCol=None, outputCol=None): """ super(IDF, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.IDF", self.uid) - self._setDefault(minDocFreq=0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1445,6 +1463,10 @@ class _ImputerParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCols, Has "The placeholder for the missing values. All occurrences of missingValue " "will be imputed.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_ImputerParams, self).__init__(*args) + self._setDefault(strategy="mean", missingValue=float("nan"), relativeError=0.001) + @since("2.2.0") def getStrategy(self): """ @@ -1582,7 +1604,6 @@ def __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, """ super(Imputer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Imputer", self.uid) - self._setDefault(strategy="mean", missingValue=float("nan"), relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1811,6 +1832,8 @@ class MaxAbsScaler(JavaEstimator, _MaxAbsScalerParams, JavaMLReadable, JavaMLWri >>> loadedModel = MaxAbsScalerModel.load(modelPath) >>> loadedModel.maxAbs == model.maxAbs True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -1951,7 +1974,6 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1): """ super(MinHashLSH, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinHashLSH", self.uid) - self._setDefault(numHashTables=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2002,6 +2024,10 @@ class _MinMaxScalerParams(HasInputCol, HasOutputCol): max = Param(Params._dummy(), "max", "Upper bound of the output feature range", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_MinMaxScalerParams, self).__init__(*args) + self._setDefault(min=0.0, max=1.0) + @since("1.6.0") def getMin(self): """ @@ -2065,6 +2091,8 @@ class MinMaxScaler(JavaEstimator, _MinMaxScalerParams, JavaMLReadable, JavaMLWri True >>> loadedModel.originalMax == model.originalMax True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -2076,7 +2104,6 @@ def __init__(self, min=0.0, max=1.0, inputCol=None, outputCol=None): """ super(MinMaxScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinMaxScaler", self.uid) - self._setDefault(min=0.0, max=1.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2209,6 +2236,8 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr >>> loadedNGram = NGram.load(ngramPath) >>> loadedNGram.getN() == ngram.getN() True + >>> loadedNGram.transform(df).take(1) == ngram.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -2289,6 +2318,8 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav >>> loadedNormalizer = Normalizer.load(normalizerPath) >>> loadedNormalizer.getP() == normalizer.getP() True + >>> loadedNormalizer.transform(df).take(1) == normalizer.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -2362,6 +2393,10 @@ class _OneHotEncoderParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCol dropLast = Param(Params._dummy(), "dropLast", "whether to drop the last category", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_OneHotEncoderParams, self).__init__(*args) + self._setDefault(handleInvalid="error", dropLast=True) + @since("2.3.0") def getDropLast(self): """ @@ -2422,6 +2457,8 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW >>> loadedModel = OneHotEncoderModel.load(modelPath) >>> loadedModel.categorySizes == model.categorySizes True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.3.0 """ @@ -2436,7 +2473,6 @@ def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropL super(OneHotEncoder, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.feature.OneHotEncoder", self.uid) - self._setDefault(handleInvalid="error", dropLast=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -2583,6 +2619,8 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead >>> loadedPx = PolynomialExpansion.load(polyExpansionPath) >>> loadedPx.getDegree() == px.getDegree() True + >>> loadedPx.transform(df).take(1) == px.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -2879,6 +2917,11 @@ class _RobustScalerParams(HasInputCol, HasOutputCol, HasRelativeError): withScaling = Param(Params._dummy(), "withScaling", "Whether to scale the data to " "quantile range", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_RobustScalerParams, self).__init__(*args) + self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True, + relativeError=0.001) + @since("3.0.0") def getLower(self): """ @@ -2954,6 +2997,8 @@ class RobustScaler(JavaEstimator, _RobustScalerParams, JavaMLReadable, JavaMLWri True >>> loadedModel.range == model.range True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.0.0 """ @@ -2967,8 +3012,6 @@ def __init__(self, lower=0.25, upper=0.75, withCentering=False, withScaling=True """ super(RobustScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RobustScaler", self.uid) - self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True, - relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -3113,6 +3156,8 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, True >>> loadedReTokenizer.getGaps() == reTokenizer.getGaps() True + >>> loadedReTokenizer.transform(df).take(1) == reTokenizer.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -3237,6 +3282,8 @@ class SQLTransformer(JavaTransformer, JavaMLReadable, JavaMLWritable): >>> loadedSqlTrans = SQLTransformer.load(sqlTransformerPath) >>> loadedSqlTrans.getStatement() == sqlTrans.getStatement() True + >>> loadedSqlTrans.transform(df).take(1) == sqlTrans.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -3291,6 +3338,10 @@ class _StandardScalerParams(HasInputCol, HasOutputCol): withStd = Param(Params._dummy(), "withStd", "Scale to unit standard deviation", typeConverter=TypeConverters.toBoolean) + def __init__(self, *args): + super(_StandardScalerParams, self).__init__(*args) + self._setDefault(withMean=False, withStd=True) + @since("1.4.0") def getWithMean(self): """ @@ -3348,6 +3399,8 @@ class StandardScaler(JavaEstimator, _StandardScalerParams, JavaMLReadable, JavaM True >>> loadedModel.mean == model.mean True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.4.0 """ @@ -3359,7 +3412,6 @@ def __init__(self, withMean=False, withStd=True, inputCol=None, outputCol=None): """ super(StandardScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.StandardScaler", self.uid) - self._setDefault(withMean=False, withStd=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -3513,6 +3565,8 @@ class StringIndexer(JavaEstimator, _StringIndexerParams, JavaMLReadable, JavaMLW >>> loadedInverter = IndexToString.load(indexToStringPath) >>> loadedInverter.getLabels() == inverter.getLabels() True + >>> loadedModel.transform(stringIndDf).take(1) == model.transform(stringIndDf).take(1) + True >>> stringIndexer.getStringOrderType() 'frequencyDesc' >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed", handleInvalid="error", @@ -3798,6 +3852,8 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, True >>> loadedRemover.getCaseSensitive() == remover.getCaseSensitive() True + >>> loadedRemover.transform(df).take(1) == remover.transform(df).take(1) + True >>> df2 = spark.createDataFrame([(["a", "b", "c"], ["a", "b"])], ["text1", "text2"]) >>> remover2 = StopWordsRemover(stopWords=["b"]) >>> remover2.setInputCols(["text1", "text2"]).setOutputCols(["words1", "words2"]) @@ -4109,6 +4165,10 @@ class _VectorIndexerParams(HasInputCol, HasOutputCol, HasHandleInvalid): "of categories of the feature).", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_VectorIndexerParams, self).__init__(*args) + self._setDefault(maxCategories=20, handleInvalid="error") + @since("1.4.0") def getMaxCategories(self): """ @@ -4189,6 +4249,8 @@ class VectorIndexer(JavaEstimator, _VectorIndexerParams, JavaMLReadable, JavaMLW True >>> loadedModel.categoryMaps == model.categoryMaps True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True >>> dfWithInvalid = spark.createDataFrame([(Vectors.dense([3.0, 1.0]),)], ["a"]) >>> indexer.getHandleInvalid() 'error' @@ -4209,7 +4271,6 @@ def __init__(self, maxCategories=20, inputCol=None, outputCol=None, handleInvali """ super(VectorIndexer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.VectorIndexer", self.uid) - self._setDefault(maxCategories=20, handleInvalid="error") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -4332,6 +4393,8 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J True >>> loadedVs.getNames() == vs.getNames() True + >>> loadedVs.transform(df).take(1) == vs.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -4431,6 +4494,11 @@ class _Word2VecParams(HasStepSize, HasMaxIter, HasSeed, HasInputCol, HasOutputCo "be divided into chunks up to the size.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_Word2VecParams, self).__init__(*args) + self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + windowSize=5, maxSentenceLength=1000) + @since("1.4.0") def getVectorSize(self): """ @@ -4524,6 +4592,8 @@ class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): True >>> loadedModel.getVectors().first().vector == model.getVectors().first().vector True + >>> loadedModel.transform(doc).take(1) == model.transform(doc).take(1) + True .. versionadded:: 1.4.0 """ @@ -4537,8 +4607,6 @@ def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, """ super(Word2Vec, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid) - self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - windowSize=5, maxSentenceLength=1000) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -4730,6 +4798,8 @@ class PCA(JavaEstimator, _PCAParams, JavaMLReadable, JavaMLWritable): True >>> loadedModel.explainedVariance == model.explainedVariance True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 1.5.0 """ @@ -4846,6 +4916,11 @@ class _RFormulaParams(HasFeaturesCol, HasLabelCol, HasHandleInvalid): "additional bucket, at index numLabels).", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_RFormulaParams, self).__init__(*args) + self._setDefault(forceIndexLabel=False, stringIndexerOrderType="frequencyDesc", + handleInvalid="error") + @since("1.5.0") def getFormula(self): """ @@ -4948,8 +5023,6 @@ def __init__(self, formula=None, featuresCol="features", labelCol="label", """ super(RFormula, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RFormula", self.uid) - self._setDefault(forceIndexLabel=False, stringIndexerOrderType="frequencyDesc", - handleInvalid="error") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5058,6 +5131,11 @@ class _SelectorParams(HasFeaturesCol, HasOutputCol, HasLabelCol): fwe = Param(Params._dummy(), "fwe", "The upper bound of the expected family-wise error rate.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_SelectorParams, self).__init__(*args) + self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, + fpr=0.05, fdr=0.05, fwe=0.05) + @since("2.1.0") def getSelectorType(self): """ @@ -5257,6 +5335,8 @@ class ANOVASelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = ANOVASelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ @@ -5272,8 +5352,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(ANOVASelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.ANOVASelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5356,6 +5434,8 @@ class ChiSqSelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = ChiSqSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -5371,8 +5451,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(ChiSqSelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.ChiSqSelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5459,6 +5537,8 @@ class FValueSelector(_Selector, JavaMLReadable, JavaMLWritable): >>> loadedModel = FValueSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ @@ -5474,8 +5554,6 @@ def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, """ super(FValueSelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.FValueSelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, - fpr=0.05, fdr=0.05, fwe=0.05) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -5652,6 +5730,8 @@ class VarianceThresholdSelector(JavaEstimator, _VarianceThresholdSelectorParams, >>> loadedModel = VarianceThresholdSelectorModel.load(modelPath) >>> loadedModel.selectedFeatures == model.selectedFeatures True + >>> loadedModel.transform(df).take(1) == model.transform(df).take(1) + True .. versionadded:: 3.1.0 """ diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index b91788a82c19a..37d3b6eec02d0 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -55,6 +55,11 @@ class _FPGrowthParams(HasPredictionCol): "but will affect the association rules generation.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_FPGrowthParams, self).__init__(*args) + self._setDefault(minSupport=0.3, minConfidence=0.8, + itemsCol="items", predictionCol="prediction") + def getItemsCol(self): """ Gets the value of itemsCol or its default value. @@ -192,6 +197,11 @@ class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): >>> new_data = spark.createDataFrame([(["t", "s"], )], ["items"]) >>> sorted(fpm.transform(new_data).first().newPrediction) ['x', 'y', 'z'] + >>> model_path = temp_path + "/fpm_model" + >>> fpm.save(model_path) + >>> model2 = FPGrowthModel.load(model_path) + >>> fpm.transform(data).take(1) == model2.transform(data).take(1) + True .. versionadded:: 2.2.0 """ @@ -204,8 +214,6 @@ def __init__(self, minSupport=0.3, minConfidence=0.8, itemsCol="items", """ super(FPGrowth, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.fpm.FPGrowth", self.uid) - self._setDefault(minSupport=0.3, minConfidence=0.8, - itemsCol="items", predictionCol="prediction") kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 99d80aa867bda..62b856046234a 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -46,6 +46,10 @@ class _ALSModelParams(HasPredictionCol, HasBlockSize): "Supported values: 'nan', 'drop'.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_ALSModelParams, self).__init__(*args) + self._setDefault(blockSize=4096) + @since("1.4.0") def getUserCol(self): """ @@ -99,6 +103,14 @@ class _ALSParams(_ALSModelParams, HasMaxIter, HasRegParam, HasCheckpointInterval "StorageLevel for ALS model factors.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_ALSParams, self).__init__(*args) + self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", + ratingCol="rating", nonnegative=False, checkpointInterval=10, + intermediateStorageLevel="MEMORY_AND_DISK", + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + @since("1.4.0") def getRank(self): """ @@ -275,6 +287,8 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): True >>> sorted(model.itemFactors.collect()) == sorted(model2.itemFactors.collect()) True + >>> model.transform(test).take(1) == model2.transform(test).take(1) + True .. versionadded:: 1.4.0 """ @@ -294,12 +308,6 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB """ super(ALS, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid) - self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", - ratingCol="rating", nonnegative=False, checkpointInterval=10, - intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", - blockSize=4096) kwargs = self._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index e82a35c8e78f1..4a8d1530b8a6f 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -104,6 +104,11 @@ class _LinearRegressionParams(_PredictorParams, HasRegParam, HasElasticNetParam, "robustness. Must be > 1.0. Only valid when loss is huber", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_LinearRegressionParams, self).__init__(*args) + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, + blockSize=1) + @since("2.3.0") def getEpsilon(self): """ @@ -185,6 +190,8 @@ class LinearRegression(_JavaRegressor, _LinearRegressionParams, JavaMLWritable, True >>> model.intercept == model2.intercept True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.numFeatures 1 >>> model.write().format("pmml").save(model_path + "_2") @@ -206,8 +213,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(LinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.LinearRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, - blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -619,6 +624,10 @@ class _IsotonicRegressionParams(HasFeaturesCol, HasLabelCol, HasPredictionCol, H "The index of the feature if featuresCol is a vector column, no effect otherwise.", typeConverter=TypeConverters.toInt) + def __init__(self, *args): + super(_IsotonicRegressionParams, self).__init__(*args) + self._setDefault(isotonic=True, featureIndex=0) + def getIsotonic(self): """ Gets the value of isotonic or its default value. @@ -668,6 +677,8 @@ class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, True >>> model.predictions == model2.predictions True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.6.0 """ @@ -681,7 +692,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(IsotonicRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.IsotonicRegression", self.uid) - self._setDefault(isotonic=True, featureIndex=0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -808,7 +818,11 @@ class _DecisionTreeRegressorParams(_DecisionTreeParams, _TreeRegressorParams, Ha .. versionadded:: 3.0.0 """ - pass + def __init__(self, *args): + super(_DecisionTreeRegressorParams, self).__init__(*args) + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="variance", leafCol="", minWeightFractionPerNode=0.0) @inherit_doc @@ -866,7 +880,8 @@ class DecisionTreeRegressor(_JavaRegressor, _DecisionTreeRegressorParams, JavaML True >>> model.transform(test1).head().variance 0.0 - + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> df3 = spark.createDataFrame([ ... (1.0, 0.2, Vectors.dense(1.0)), ... (1.0, 0.8, Vectors.dense(1.0)), @@ -895,9 +910,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(DecisionTreeRegressor, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.DecisionTreeRegressor", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="variance", leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1052,7 +1064,14 @@ class _RandomForestRegressorParams(_RandomForestParams, _TreeRegressorParams): .. versionadded:: 3.0.0 """ - pass + + def __init__(self, *args): + super(_RandomForestRegressorParams, self).__init__(*args) + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, + impurity="variance", subsamplingRate=1.0, numTrees=20, + featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, + bootstrap=True) @inherit_doc @@ -1113,6 +1132,8 @@ class RandomForestRegressor(_JavaRegressor, _RandomForestRegressorParams, JavaML >>> model2 = RandomForestRegressionModel.load(model_path) >>> model.featureImportances == model2.featureImportances True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 1.4.0 """ @@ -1135,11 +1156,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(RandomForestRegressor, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.RandomForestRegressor", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="variance", subsamplingRate=1.0, numTrees=20, - featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, - bootstrap=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1310,6 +1326,14 @@ class _GBTRegressorParams(_GBTParams, _TreeRegressorParams): "Supported options: " + ", ".join(supportedLossTypes), typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_GBTRegressorParams, self).__init__(*args) + self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, + maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, + checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, + impurity="variance", featureSubsetStrategy="all", validationTol=0.01, + leafCol="", minWeightFractionPerNode=0.0) + @since("1.4.0") def getLossType(self): """ @@ -1373,6 +1397,8 @@ class GBTRegressor(_JavaRegressor, _GBTRegressorParams, JavaMLWritable, JavaMLRe True >>> model.treeWeights == model2.treeWeights True + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True >>> model.trees [DecisionTreeRegressionModel...depth=..., DecisionTreeRegressionModel...] >>> validation = spark.createDataFrame([(0.0, Vectors.dense(-1.0))], @@ -1407,11 +1433,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred """ super(GBTRegressor, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.regression.GBTRegressor", self.uid) - self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, - checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, - impurity="variance", featureSubsetStrategy="all", validationTol=0.01, - leafCol="", minWeightFractionPerNode=0.0) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1630,6 +1651,12 @@ class _AFTSurvivalRegressionParams(_PredictorParams, HasMaxIter, HasTol, HasFitI "corresponding quantileProbabilities if it is set.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_AFTSurvivalRegressionParams, self).__init__(*args) + self._setDefault(censorCol="censor", + quantileProbabilities=[0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99], + maxIter=100, tol=1E-6, blockSize=1) + @since("1.6.0") def getCensorCol(self): """ @@ -1704,6 +1731,8 @@ class AFTSurvivalRegression(_JavaRegressor, _AFTSurvivalRegressionParams, True >>> model.scale == model2.scale True + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 1.6.0 """ @@ -1722,9 +1751,6 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred super(AFTSurvivalRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.AFTSurvivalRegression", self.uid) - self._setDefault(censorCol="censor", - quantileProbabilities=[0.01, 0.05, 0.1, 0.25, 0.5, 0.75, 0.9, 0.95, 0.99], - maxIter=100, tol=1E-6, blockSize=1) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1891,8 +1917,8 @@ class _GeneralizedLinearRegressionParams(_PredictorParams, HasFitIntercept, HasM "or empty, we treat all instance offsets as 0.0", typeConverter=TypeConverters.toString) - def __init__(self): - super(_GeneralizedLinearRegressionParams, self).__init__() + def __init__(self, *args): + super(_GeneralizedLinearRegressionParams, self).__init__(*args) self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", variancePower=0.0, aggregationDepth=2) @@ -2010,6 +2036,8 @@ class GeneralizedLinearRegression(_JavaRegressor, _GeneralizedLinearRegressionPa True >>> model.coefficients[0] == model2.coefficients[0] True + >>> model.transform(df).take(1) == model2.transform(df).take(1) + True .. versionadded:: 2.0.0 """ @@ -2376,7 +2404,7 @@ def __repr__(self): class _FactorizationMachinesParams(_PredictorParams, HasMaxIter, HasStepSize, HasTol, - HasSolver, HasSeed, HasFitIntercept, HasRegParam): + HasSolver, HasSeed, HasFitIntercept, HasRegParam, HasWeightCol): """ Params for :py:class:`FMRegressor`, :py:class:`FMRegressionModel`, :py:class:`FMClassifier` and :py:class:`FMClassifierModel`. @@ -2401,8 +2429,8 @@ class _FactorizationMachinesParams(_PredictorParams, HasMaxIter, HasStepSize, Ha solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) - def __init__(self): - super(_FactorizationMachinesParams, self).__init__() + def __init__(self, *args): + super(_FactorizationMachinesParams, self).__init__(*args) self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, tol=1e-6, solver="adamW") @@ -2480,6 +2508,17 @@ class FMRegressor(_JavaRegressor, _FactorizationMachinesParams, JavaMLWritable, DenseVector([0.9978]) >>> model.factors DenseMatrix(1, 2, [0.0173, 0.0021], 1) + >>> model_path = temp_path + "/fm_model" + >>> model.save(model_path) + >>> model2 = FMRegressionModel.load(model_path) + >>> model2.intercept + -0.0032501766849261557 + >>> model2.linear + DenseVector([0.9978]) + >>> model2.factors + DenseMatrix(1, 2, [0.0173, 0.0021], 1) + >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + True .. versionadded:: 3.0.0 """ diff --git a/python/pyspark/ml/stat.py b/python/pyspark/ml/stat.py index 70de8425613ec..517c984252768 100644 --- a/python/pyspark/ml/stat.py +++ b/python/pyspark/ml/stat.py @@ -222,14 +222,14 @@ class Summarizer(object): +-----------------------------------+ |aggregate_metrics(features, weight)| +-----------------------------------+ - |[[1.0,1.0,1.0], 1] | + |{[1.0,1.0,1.0], 1} | +-----------------------------------+ >>> df.select(summarizer.summary(df.features)).show(truncate=False) +--------------------------------+ |aggregate_metrics(features, 1.0)| +--------------------------------+ - |[[1.0,1.5,2.0], 2] | + |{[1.0,1.5,2.0], 2} | +--------------------------------+ >>> df.select(Summarizer.mean(df.features, df.weight)).show(truncate=False) diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index e1abd59a2d7b2..44731568b6d1f 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -359,16 +359,14 @@ def test_java_params(self): and issubclass(cls, JavaParams) and not inspect.isabstract(cls) \ and not re.match("_?Java", name) and name != '_LSH' \ and name != '_Selector': - # NOTE: disable check_params_exist until there is parity with Scala API - - check_params(self, cls(), check_params_exist=False) + check_params(self, cls(), check_params_exist=True) # Additional classes that need explicit construction from pyspark.ml.feature import CountVectorizerModel, StringIndexerModel check_params(self, CountVectorizerModel.from_vocabulary(['a'], 'input'), - check_params_exist=False) + check_params_exist=True) check_params(self, StringIndexerModel.from_labels(['a', 'b'], 'input'), - check_params_exist=False) + check_params_exist=True) if __name__ == "__main__": diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index d305be8b96cd4..6b05ffaa7d52f 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -18,11 +18,12 @@ import sys import unittest -from pyspark.ml.classification import BinaryLogisticRegressionSummary, FMClassifier, \ +from pyspark.ml.classification import BinaryLogisticRegressionSummary, \ + BinaryRandomForestClassificationSummary, FMClassifier, \ FMClassificationSummary, LinearSVC, LinearSVCSummary, \ - BinaryRandomForestClassificationSummary, LogisticRegression, \ - LogisticRegressionSummary, RandomForestClassificationSummary, \ - RandomForestClassifier + LogisticRegression, LogisticRegressionSummary, \ + MultilayerPerceptronClassifier, MultilayerPerceptronClassificationSummary, \ + RandomForestClassificationSummary, RandomForestClassifier from pyspark.ml.clustering import BisectingKMeans, GaussianMixture, KMeans from pyspark.ml.linalg import Vectors from pyspark.ml.regression import GeneralizedLinearRegression, LinearRegression @@ -354,6 +355,42 @@ def test_fm_classification_summary(self): self.assertTrue(isinstance(sameSummary, FMClassificationSummary)) self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + def test_mlp_classification_summary(self): + df = self.spark.createDataFrame([(0.0, Vectors.dense([0.0, 0.0])), + (1.0, Vectors.dense([0.0, 1.0])), + (1.0, Vectors.dense([1.0, 0.0])), + (0.0, Vectors.dense([1.0, 1.0])) + ], + ["label", "features"]) + mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], seed=123) + model = mlp.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary() + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.predictionCol, "prediction") + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertAlmostEqual(s.accuracy, 1.0, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) + self.assertAlmostEqual(s.weightedRecall, 1.0, 2) + self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, MultilayerPerceptronClassificationSummary)) + self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) + def test_gaussian_mixture_summary(self): data = [(Vectors.dense(1.0),), (Vectors.dense(5.0),), (Vectors.dense(10.0),), (Vectors.sparse(1, [], []),)] diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 7f3d942e2e456..d7800e0c9020e 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -206,6 +206,10 @@ class _CrossValidatorParams(_ValidatorParams): "with range [0, numFolds) and Spark will throw exception on out-of-range " + "fold numbers.", typeConverter=TypeConverters.toString) + def __init__(self, *args): + super(_CrossValidatorParams, self).__init__(*args) + self._setDefault(numFolds=3, foldCol="") + @since("1.4.0") def getNumFolds(self): """ @@ -262,6 +266,8 @@ class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollec [0.5, ... >>> evaluator.evaluate(cvModel.transform(dataset)) 0.8333... + >>> evaluator.evaluate(cvModelRead.transform(dataset)) + 0.8333... .. versionadded:: 1.4.0 """ @@ -274,7 +280,7 @@ def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numF seed=None, parallelism=1, collectSubModels=False, foldCol="") """ super(CrossValidator, self).__init__() - self._setDefault(numFolds=3, parallelism=1, foldCol="") + self._setDefault(parallelism=1) kwargs = self._input_kwargs self._set(**kwargs) @@ -600,6 +606,10 @@ class _TrainValidationSplitParams(_ValidatorParams): trainRatio = Param(Params._dummy(), "trainRatio", "Param for ratio between train and\ validation data. Must be between 0 and 1.", typeConverter=TypeConverters.toFloat) + def __init__(self, *args): + super(_TrainValidationSplitParams, self).__init__(*args) + self._setDefault(trainRatio=0.75) + @since("2.0.0") def getTrainRatio(self): """ @@ -645,8 +655,11 @@ class TrainValidationSplit(Estimator, _TrainValidationSplitParams, HasParallelis [0.5, ... >>> evaluator.evaluate(tvsModel.transform(dataset)) 0.833... + >>> evaluator.evaluate(tvsModelRead.transform(dataset)) + 0.833... .. versionadded:: 2.0.0 + """ @keyword_only @@ -657,7 +670,7 @@ def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, trai parallelism=1, collectSubModels=False, seed=None) """ super(TrainValidationSplit, self).__init__() - self._setDefault(trainRatio=0.75, parallelism=1) + self._setDefault(parallelism=1) kwargs = self._input_kwargs self._set(**kwargs) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 437b2c446529a..4ee486800f882 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -859,12 +859,18 @@ def collect(self): def collectWithJobGroup(self, groupId, description, interruptOnCancel=False): """ - .. note:: Experimental - When collect rdd, use this method to specify job group. + .. note:: Deprecated in 3.1.0. Use :class:`pyspark.InheritableThread` with + the pinned thread mode enabled. + .. versionadded:: 3.0.0 """ + warnings.warn( + "Deprecated in 3.1, Use pyspark.InheritableThread with " + "the pinned thread mode enabled.", + DeprecationWarning) + with SCCallSiteSync(self.context) as css: sock_info = self.ctx._jvm.PythonRDD.collectAndServeWithJobGroup( self._jrdd.rdd(), groupId, description, interruptOnCancel) diff --git a/python/pyspark/resource/requests.py b/python/pyspark/resource/requests.py index 56ad6e8be9bcb..61491080a8c43 100644 --- a/python/pyspark/resource/requests.py +++ b/python/pyspark/resource/requests.py @@ -91,6 +91,7 @@ class ExecutorResourceRequests(object): _MEMORY = "memory" _OVERHEAD_MEM = "memoryOverhead" _PYSPARK_MEM = "pyspark.memory" + _OFFHEAP_MEM = "offHeap" def __init__(self, _jvm=None, _requests=None): from pyspark import SparkContext @@ -139,6 +140,14 @@ def pysparkMemory(self, amount): ExecutorResourceRequest(self._PYSPARK_MEM, _parse_memory(amount)) return self + def offheapMemory(self, amount): + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.offHeapMemory(amount) + else: + self._executor_resources[self._OFFHEAP_MEM] = \ + ExecutorResourceRequest(self._OFFHEAP_MEM, _parse_memory(amount)) + return self + def cores(self, amount): if self._java_executor_resource_requests is not None: self._java_executor_resource_requests.cores(amount) diff --git a/python/pyspark/resource/tests/test_resources.py b/python/pyspark/resource/tests/test_resources.py index 9eb5a35e58df5..09c0d3ca20703 100644 --- a/python/pyspark/resource/tests/test_resources.py +++ b/python/pyspark/resource/tests/test_resources.py @@ -25,15 +25,16 @@ class ResourceProfileTests(unittest.TestCase): def test_profile_before_sc(self): rpb = ResourceProfileBuilder() ereqs = ExecutorResourceRequests().cores(2).memory("6g").memoryOverhead("1g") - ereqs.pysparkMemory("2g").resource("gpu", 2, "testGpus", "nvidia.com") + ereqs.pysparkMemory("2g").offheapMemory("3g").resource("gpu", 2, "testGpus", "nvidia.com") treqs = TaskResourceRequests().cpus(2).resource("gpu", 2) def assert_request_contents(exec_reqs, task_reqs): - self.assertEqual(len(exec_reqs), 5) + self.assertEqual(len(exec_reqs), 6) self.assertEqual(exec_reqs["cores"].amount, 2) self.assertEqual(exec_reqs["memory"].amount, 6144) self.assertEqual(exec_reqs["memoryOverhead"].amount, 1024) self.assertEqual(exec_reqs["pyspark.memory"].amount, 2048) + self.assertEqual(exec_reqs["offHeap"].amount, 3072) self.assertEqual(exec_reqs["gpu"].amount, 2) self.assertEqual(exec_reqs["gpu"].discoveryScript, "testGpus") self.assertEqual(exec_reqs["gpu"].resourceName, "gpu") diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 5d2d63850e9b2..308305e5f46c6 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -25,7 +25,7 @@ import random import sys -import pyspark.heapq3 as heapq +import heapq from pyspark.serializers import BatchedSerializer, PickleSerializer, FlattenedValuesSerializer, \ CompressedSerializer, AutoBatchedSerializer from pyspark.util import fail_on_stopiteration @@ -498,7 +498,7 @@ def load(f): if current_chunk: chunks.append(iter(current_chunk)) - return heapq.merge(chunks, key=key, reverse=reverse) + return heapq.merge(*chunks, key=key, reverse=reverse) class ExternalList(object): @@ -796,7 +796,7 @@ def load_partition(j): if self._sorted: # all the partitions are already sorted - sorted_items = heapq.merge(disk_items, key=operator.itemgetter(0)) + sorted_items = heapq.merge(*disk_items, key=operator.itemgetter(0)) else: # Flatten the combined values, so it will not consume huge diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 023fbeabcbabc..1027918adbe15 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -29,7 +29,7 @@ from pyspark.traceback_utils import SCCallSiteSync from pyspark.sql.types import _parse_datatype_json_string from pyspark.sql.column import Column, _to_seq, _to_list, _to_java_column -from pyspark.sql.readwriter import DataFrameWriter +from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.streaming import DataStreamWriter from pyspark.sql.types import * from pyspark.sql.pandas.conversion import PandasConversionMixin @@ -2240,6 +2240,22 @@ def inputFiles(self): sinceversion=1.4, doc=":func:`drop_duplicates` is an alias for :func:`dropDuplicates`.") + @since(3.1) + def writeTo(self, table): + """ + Create a write configuration builder for v2 sources. + + This builder is used to configure and execute write operations. + + For example, to append or create or replace existing tables. + + >>> df.writeTo("catalog.db.table").append() # doctest: +SKIP + >>> df.writeTo( # doctest: +SKIP + ... "catalog.db.table" + ... ).partitionedBy("col").createOrReplace() + """ + return DataFrameWriterV2(self, table) + def _to_scala_map(sc, jm): """ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 5a352104c4eca..308642b136f75 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1196,6 +1196,8 @@ def to_date(col, format=None): By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format is omitted. Equivalent to ``col.cast("date")``. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_date(df.t).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] @@ -1219,6 +1221,8 @@ def to_timestamp(col, format=None): By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format is omitted. Equivalent to ``col.cast("timestamp")``. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_timestamp(df.t).alias('dt')).collect() [Row(dt=datetime.datetime(1997, 2, 28, 10, 30))] @@ -1992,7 +1996,7 @@ def map_from_arrays(col1, col2): +----------------+ | map| +----------------+ - |[2 -> a, 5 -> b]| + |{2 -> a, 5 -> b}| +----------------+ """ sc = SparkContext._active_spark_context @@ -2068,7 +2072,11 @@ def slice(x, start, length): [Row(sliced=[2, 3]), Row(sliced=[5])] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.slice(_to_java_column(x), start, length)) + return Column(sc._jvm.functions.slice( + _to_java_column(x), + start._jc if isinstance(start, Column) else start, + length._jc if isinstance(length, Column) else length + )) @since(2.4) @@ -2308,9 +2316,9 @@ def explode_outer(col): +---+----------+----+ | id| a_map| col| +---+----------+----+ - | 1|[x -> 1.0]| foo| - | 1|[x -> 1.0]| bar| - | 2| []|null| + | 1|{x -> 1.0}| foo| + | 1|{x -> 1.0}| bar| + | 2| {}|null| | 3| null|null| +---+----------+----+ """ @@ -2343,9 +2351,9 @@ def posexplode_outer(col): +---+----------+----+----+ | id| a_map| pos| col| +---+----------+----+----+ - | 1|[x -> 1.0]| 0| foo| - | 1|[x -> 1.0]| 1| bar| - | 2| []|null|null| + | 1|{x -> 1.0}| 0| foo| + | 1|{x -> 1.0}| 1| bar| + | 2| {}|null|null| | 3| null|null|null| +---+----------+----+----+ """ @@ -2742,7 +2750,7 @@ def map_entries(col): +----------------+ | entries| +----------------+ - |[[1, a], [2, b]]| + |[{1, a}, {2, b}]| +----------------+ """ sc = SparkContext._active_spark_context @@ -2762,7 +2770,7 @@ def map_from_entries(col): +----------------+ | map| +----------------+ - |[1 -> a, 2 -> b]| + |{1 -> a, 2 -> b}| +----------------+ """ sc = SparkContext._active_spark_context @@ -2814,7 +2822,7 @@ def map_concat(*cols): +------------------------+ |map3 | +------------------------+ - |[1 -> a, 2 -> b, 3 -> c]| + |{1 -> a, 2 -> b, 3 -> c}| +------------------------+ """ sc = SparkContext._active_spark_context @@ -3233,7 +3241,7 @@ def transform_keys(col, f): +-------------------------+ |data_upper | +-------------------------+ - |[BAR -> 2.0, FOO -> -2.0]| + |{BAR -> 2.0, FOO -> -2.0}| +-------------------------+ """ return _invoke_higher_order_function("TransformKeys", [col], [f]) @@ -3260,7 +3268,7 @@ def transform_values(col, f): +---------------------------------------+ |new_data | +---------------------------------------+ - |[OPS -> 34.0, IT -> 20.0, SALES -> 2.0]| + |{OPS -> 34.0, IT -> 20.0, SALES -> 2.0}| +---------------------------------------+ """ return _invoke_higher_order_function("TransformValues", [col], [f]) @@ -3286,7 +3294,7 @@ def map_filter(col, f): +--------------------------+ |data_filtered | +--------------------------+ - |[baz -> 32.0, foo -> 42.0]| + |{baz -> 32.0, foo -> 42.0}| +--------------------------+ """ return _invoke_higher_order_function("MapFilter", [col], [f]) @@ -3316,12 +3324,124 @@ def map_zip_with(col1, col2, f): +---------------------------+ |updated_data | +---------------------------+ - |[SALES -> 16.8, IT -> 48.0]| + |{SALES -> 16.8, IT -> 48.0}| +---------------------------+ """ return _invoke_higher_order_function("MapZipWith", [col1, col2], [f]) +# ---------------------- Partition transform functions -------------------------------- + +@since(3.1) +def years(col): + """ + Partition transform function: A transform for timestamps and dates + to partition data into years. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... years("ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.years(_to_java_column(col))) + + +@since(3.1) +def months(col): + """ + Partition transform function: A transform for timestamps and dates + to partition data into months. + + >>> df.writeTo("catalog.db.table").partitionedBy( + ... months("ts") + ... ).createOrReplace() # doctest: +SKIP + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.months(_to_java_column(col))) + + +@since(3.1) +def days(col): + """ + Partition transform function: A transform for timestamps and dates + to partition data into days. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... days("ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.days(_to_java_column(col))) + + +@since(3.1) +def hours(col): + """ + Partition transform function: A transform for timestamps + to partition data into hours. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... hours("ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.hours(_to_java_column(col))) + + +@since(3.1) +def bucket(numBuckets, col): + """ + Partition transform function: A transform for any type that partitions + by a hash of the input column. + + >>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP + ... bucket(42, "ts") + ... ).createOrReplace() + + .. warning:: + This function can be used only in combinatiion with + :py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy` + method of the `DataFrameWriterV2`. + + """ + if not isinstance(numBuckets, (int, Column)): + raise TypeError( + "numBuckets should be a Column or and int, got {}".format(type(numBuckets)) + ) + + sc = SparkContext._active_spark_context + numBuckets = ( + _create_column_from_literal(numBuckets) + if isinstance(numBuckets, int) + else _to_java_column(numBuckets) + ) + return Column(sc._jvm.functions.bucket(numBuckets, _to_java_column(col))) + + # ---------------------------- User Defined Function ---------------------------------- @since(1.3) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index a83aece2e485d..8560ef78c1fb0 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -18,7 +18,7 @@ from py4j.java_gateway import JavaClass from pyspark import RDD, since -from pyspark.sql.column import _to_seq +from pyspark.sql.column import _to_seq, _to_java_column from pyspark.sql.types import * from pyspark.sql import utils from pyspark.sql.utils import to_str @@ -184,7 +184,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, samplingRatio=None, dropFieldIfAllNull=None, encoding=None, locale=None, pathGlobFilter=None, - recursiveFileLookup=None): + recursiveFileLookup=None, allowNonNumericNumbers=None): """ Loads JSON files and returns the results as a :class:`DataFrame`. @@ -264,6 +264,14 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + :param allowNonNumericNumbers: allows JSON parser to recognize set of "Not-a-Number" (NaN) + tokens as legal floating number values. If None is set, + it uses the default value, ``true``. + + * ``+INF``: for positive infinity, as well as alias of + ``+Infinity`` and ``Infinity``. + * ``-INF``: for negative infinity, alias ``-Infinity``. + * ``NaN``: for other not-a-numbers, like result of division by zero. .. _partition discovery: https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery @@ -287,7 +295,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, samplingRatio=samplingRatio, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, - locale=locale, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) + locale=locale, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, + allowNonNumericNumbers=allowNonNumericNumbers) if isinstance(path, str): path = [path] if type(path) == list: @@ -335,6 +344,9 @@ def parquet(self, *paths, **options): :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned') >>> df.dtypes [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] @@ -367,6 +379,9 @@ def text(self, paths, wholetext=False, lineSep=None, pathGlobFilter=None, :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> df = spark.read.text('python/test_support/sql/text-test.txt') >>> df.collect() [Row(value='hello'), Row(value='this')] @@ -502,6 +517,10 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df = spark.read.csv('python/test_support/sql/ages.csv') >>> df.dtypes [('_c0', 'string'), ('_c1', 'string')] @@ -561,6 +580,9 @@ def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=N :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> df = spark.read.orc('python/test_support/sql/orc_partitioned') >>> df.dtypes [('a', 'bigint'), ('b', 'int'), ('c', 'int')] @@ -893,6 +915,8 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm :param ignoreNullFields: Whether to ignore null fields when generating JSON objects. If None is set, it uses the default value, ``true``. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) @@ -1007,6 +1031,8 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No :param lineSep: defines the line separator that should be used for writing. If None is set, it uses the default value, ``\\n``. Maximum length is 1 character. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) @@ -1075,6 +1101,145 @@ def jdbc(self, url, table, mode=None, properties=None): self.mode(mode)._jwrite.jdbc(url, table, jprop) +class DataFrameWriterV2(object): + """ + Interface used to write a class:`pyspark.sql.dataframe.DataFrame` + to external storage using the v2 API. + + .. versionadded:: 3.1.0 + """ + + def __init__(self, df, table): + self._df = df + self._spark = df.sql_ctx + self._jwriter = df._jdf.writeTo(table) + + @since(3.1) + def using(self, provider): + """ + Specifies a provider for the underlying output data source. + Spark's default catalog supports "parquet", "json", etc. + """ + self._jwriter.using(provider) + return self + + @since(3.1) + def option(self, key, value): + """ + Add a write option. + """ + self._jwriter.option(key, to_str(value)) + return self + + @since(3.1) + def options(self, **options): + """ + Add write options. + """ + options = {k: to_str(v) for k, v in options.items()} + self._jwriter.options(options) + return self + + @since(3.1) + def tableProperty(self, property, value): + """ + Add table property. + """ + self._jwriter.tableProperty(property, value) + return self + + @since(3.1) + def partitionedBy(self, col, *cols): + """ + Partition the output table created by `create`, `createOrReplace`, or `replace` using + the given columns or transforms. + + When specified, the table data will be stored by these values for efficient reads. + + For example, when a table is partitioned by day, it may be stored + in a directory layout like: + + * `table/day=2019-06-01/` + * `table/day=2019-06-02/` + + Partitioning is one of the most widely used techniques to optimize physical data layout. + It provides a coarse-grained index for skipping unnecessary data reads when queries have + predicates on the partitioned columns. In order for partitioning to work well, the number + of distinct values in each column should typically be less than tens of thousands. + + `col` and `cols` support only the following functions: + + * :py:func:`pyspark.sql.functions.years` + * :py:func:`pyspark.sql.functions.months` + * :py:func:`pyspark.sql.functions.days` + * :py:func:`pyspark.sql.functions.hours` + * :py:func:`pyspark.sql.functions.bucket` + + """ + col = _to_java_column(col) + cols = _to_seq(self._spark._sc, [_to_java_column(c) for c in cols]) + return self + + @since(3.1) + def create(self): + """ + Create a new table from the contents of the data frame. + + The new table's schema, partition layout, properties, and other configuration will be + based on the configuration set on this writer. + """ + self._jwriter.create() + + @since(3.1) + def replace(self): + """ + Replace an existing table with the contents of the data frame. + + The existing table's schema, partition layout, properties, and other configuration will be + replaced with the contents of the data frame and the configuration set on this writer. + """ + self._jwriter.replace() + + @since(3.1) + def createOrReplace(self): + """ + Create a new table or replace an existing table with the contents of the data frame. + + The output table's schema, partition layout, properties, + and other configuration will be based on the contents of the data frame + and the configuration set on this writer. + If the table exists, its configuration and data will be replaced. + """ + self._jwriter.createOrReplace() + + @since(3.1) + def append(self): + """ + Append the contents of the data frame to the output table. + """ + self._jwriter.append() + + @since(3.1) + def overwrite(self, condition): + """ + Overwrite rows matching the given filter condition with the contents of the data frame in + the output table. + """ + condition = _to_java_column(column) + self._jwriter.overwrite(condition) + + @since(3.1) + def overwritePartitions(self): + """ + Overwrite all partition for which the data frame contains at least one row with the contents + of the data frame in the output table. + + This operation is equivalent to Hive's `INSERT OVERWRITE ... PARTITION`, which replaces + partitions dynamically depending on the contents of the data frame. + """ + self._jwriter.overwritePartitions() + + def _test(): import doctest import os diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 5c528c1d54df7..b1832f420e389 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -420,7 +420,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, locale=None, dropFieldIfAllNull=None, encoding=None, pathGlobFilter=None, - recursiveFileLookup=None): + recursiveFileLookup=None, allowNonNumericNumbers=None): """ Loads a JSON file stream and returns the results as a :class:`DataFrame`. @@ -500,6 +500,14 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + :param allowNonNumericNumbers: allows JSON parser to recognize set of "Not-a-Number" (NaN) + tokens as legal floating number values. If None is set, + it uses the default value, ``true``. + + * ``+INF``: for positive infinity, as well as alias of + ``+Infinity`` and ``Infinity``. + * ``-INF``: for negative infinity, alias ``-Infinity``. + * ``NaN``: for other not-a-numbers, like result of division by zero. .. _partition discovery: https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery @@ -520,7 +528,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, - pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) + pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, + allowNonNumericNumbers=allowNonNumericNumbers) if isinstance(path, str): return self._df(self._jreader.json(path)) else: @@ -541,6 +550,9 @@ def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=N :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp()) >>> orc_sdf.isStreaming True @@ -571,6 +583,9 @@ def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLook :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp()) >>> parquet_sdf.isStreaming True @@ -607,6 +622,9 @@ def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None, :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + >>> text_sdf = spark.readStream.text(tempfile.mkdtemp()) >>> text_sdf.isStreaming True @@ -737,6 +755,10 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html + >>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema) >>> csv_sdf.isStreaming True diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 7dcc19f3ba45d..02180daf081ec 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -292,6 +292,16 @@ def test_input_file_name_reset_for_rdd(self): for result in results: self.assertEqual(result[0], '') + def test_slice(self): + from pyspark.sql.functions import slice, lit + + df = self.spark.createDataFrame([([1, 2, 3],), ([4, 5],)], ['x']) + + self.assertEquals( + df.select(slice(df.x, 2, 2).alias("sliced")).collect(), + df.select(slice(df.x, lit(2), lit(2)).alias("sliced")).collect(), + ) + def test_array_repeat(self): from pyspark.sql.functions import array_repeat, lit diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index 2530cc2ebf224..8e34d3865c9d8 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -19,6 +19,8 @@ import shutil import tempfile +from pyspark.sql.functions import col +from pyspark.sql.readwriter import DataFrameWriterV2 from pyspark.sql.types import * from pyspark.testing.sqlutils import ReusedSQLTestCase @@ -163,6 +165,40 @@ def test_insert_into(self): self.assertEqual(6, self.spark.sql("select * from test_table").count()) +class ReadwriterV2Tests(ReusedSQLTestCase): + def test_api(self): + df = self.df + writer = df.writeTo("testcat.t") + self.assertIsInstance(writer, DataFrameWriterV2) + self.assertIsInstance(writer.option("property", "value"), DataFrameWriterV2) + self.assertIsInstance(writer.options(property="value"), DataFrameWriterV2) + self.assertIsInstance(writer.using("source"), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy("id"), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(col("id")), DataFrameWriterV2) + self.assertIsInstance(writer.tableProperty("foo", "bar"), DataFrameWriterV2) + + def test_partitioning_functions(self): + import datetime + from pyspark.sql.functions import years, months, days, hours, bucket + + df = self.spark.createDataFrame( + [(1, datetime.datetime(2000, 1, 1), "foo")], + ("id", "ts", "value") + ) + + writer = df.writeTo("testcat.t") + + self.assertIsInstance(writer.partitionedBy(years("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(months("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(days("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(hours("ts")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(bucket(11, "id")), DataFrameWriterV2) + self.assertIsInstance(writer.partitionedBy(bucket(11, col("id"))), DataFrameWriterV2) + self.assertIsInstance( + writer.partitionedBy(bucket(3, "id"), hours(col("ts"))), DataFrameWriterV2 + ) + + if __name__ == "__main__": import unittest from pyspark.sql.tests.test_readwriter import * diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 168299e385e78..f398cec344725 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -275,6 +275,17 @@ def test_disallow_to_create_spark_context_in_executors(self): self.assertIn("SparkContext should only be created and accessed on the driver.", str(context.exception)) + def test_allow_to_create_spark_context_in_executors(self): + # SPARK-32160: SparkContext can be created in executors if the config is set. + + def create_spark_context(): + conf = SparkConf().set("spark.executor.allowSparkContext", "true") + with SparkContext(conf=conf): + pass + + with SparkContext("local-cluster[3, 1, 1024]") as sc: + sc.range(2).foreach(lambda _: create_spark_context()) + class ContextTestsWithResources(unittest.TestCase): diff --git a/python/pyspark/tests/test_pin_thread.py b/python/pyspark/tests/test_pin_thread.py index 657d129fe63bb..50eb8e0ec8b50 100644 --- a/python/pyspark/tests/test_pin_thread.py +++ b/python/pyspark/tests/test_pin_thread.py @@ -20,7 +20,7 @@ import threading import unittest -from pyspark import SparkContext, SparkConf +from pyspark import SparkContext, SparkConf, InheritableThread class PinThreadTests(unittest.TestCase): @@ -143,6 +143,27 @@ def run_job(job_group, index): is_job_cancelled[i], "Thread {i}: Job in group B did not succeeded.".format(i=i)) + def test_inheritable_local_property(self): + self.sc.setLocalProperty("a", "hi") + expected = [] + + def get_inner_local_prop(): + expected.append(self.sc.getLocalProperty("b")) + + def get_outer_local_prop(): + expected.append(self.sc.getLocalProperty("a")) + self.sc.setLocalProperty("b", "hello") + t2 = InheritableThread(target=get_inner_local_prop) + t2.start() + t2.join() + + t1 = InheritableThread(target=get_outer_local_prop) + t1.start() + t1.join() + + self.assertEqual(self.sc.getLocalProperty("b"), None) + self.assertEqual(expected, ["hi", "hello"]) + if __name__ == "__main__": import unittest diff --git a/python/pyspark/util.py b/python/pyspark/util.py index c003586e9c03b..86e5ab5a01585 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -16,10 +16,13 @@ # limitations under the License. # +import threading import re import sys import traceback +from py4j.clientserver import ClientServer + __all__ = [] @@ -114,6 +117,64 @@ def _parse_memory(s): raise ValueError("invalid format: " + s) return int(float(s[:-1]) * units[s[-1].lower()]) + +class InheritableThread(threading.Thread): + """ + Thread that is recommended to be used in PySpark instead of :class:`threading.Thread` + when the pinned thread mode is enabled. The usage of this class is exactly same as + :class:`threading.Thread` but correctly inherits the inheritable properties specific + to JVM thread such as ``InheritableThreadLocal``. + + Also, note that pinned thread mode does not close the connection from Python + to JVM when the thread is finished in the Python side. With this class, Python + garbage-collects the Python thread instance and also closes the connection + which finishes JVM thread correctly. + + When the pinned thread mode is off, this works as :class:`threading.Thread`. + + .. note:: Experimental + + .. versionadded:: 3.1.0 + """ + def __init__(self, target, *args, **kwargs): + from pyspark import SparkContext + + sc = SparkContext._active_spark_context + + if isinstance(sc._gateway, ClientServer): + # Here's when the pinned-thread mode (PYSPARK_PIN_THREAD) is on. + properties = sc._jsc.sc().getLocalProperties().clone() + self._sc = sc + + def copy_local_properties(*a, **k): + sc._jsc.sc().setLocalProperties(properties) + return target(*a, **k) + + super(InheritableThread, self).__init__( + target=copy_local_properties, *args, **kwargs) + else: + super(InheritableThread, self).__init__(target=target, *args, **kwargs) + + def __del__(self): + from pyspark import SparkContext + + if isinstance(SparkContext._gateway, ClientServer): + thread_connection = self._sc._jvm._gateway_client.thread_connection.connection() + if thread_connection is not None: + connections = self._sc._jvm._gateway_client.deque + + # Reuse the lock for Py4J in PySpark + with SparkContext._lock: + for i in range(len(connections)): + if connections[i] is thread_connection: + connections[i].close() + del connections[i] + break + else: + # Just in case the connection was not closed but removed from the queue. + thread_connection.close() + + if __name__ == "__main__": import doctest (failure_count, test_count) = doctest.testmod() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 9b54affb137f5..84e5cca5d3c00 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -468,7 +468,7 @@ def main(infile, outfile): version = utf8_deserializer.loads(infile) if version != "%d.%d" % sys.version_info[:2]: raise Exception(("Python in worker has different version %s than that in " + - "driver %s, PySpark cannot run with different minor versions." + + "driver %s, PySpark cannot run with different minor versions. " + "Please check environment variables PYSPARK_PYTHON and " + "PYSPARK_DRIVER_PYTHON are correctly set.") % ("%d.%d" % sys.version_info[:2], version)) diff --git a/python/setup.py b/python/setup.py index c456a32fea87c..6a96ee743b597 100755 --- a/python/setup.py +++ b/python/setup.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -161,6 +161,7 @@ def _supports_symlinks(): author_email='dev@spark.apache.org', url='https://github.com/apache/spark/tree/master/python', packages=['pyspark', + 'pyspark.cloudpickle', 'pyspark.mllib', 'pyspark.mllib.linalg', 'pyspark.mllib.stat', diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index b4155fed8aa24..713d35dcf64f5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -21,9 +21,11 @@ import java.util.{Collections, UUID} import java.util.Properties import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.{KubernetesClient, Watch} +import io.fabric8.kubernetes.client.Watcher.Action import scala.collection.mutable import scala.util.control.NonFatal +import util.control.Breaks._ import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkApplication @@ -127,25 +129,37 @@ private[spark] class Client( .endSpec() .build() val driverPodName = resolvedDriverPod.getMetadata.getName - Utils.tryWithResource( - kubernetesClient - .pods() - .withName(driverPodName) - .watch(watcher)) { _ => - val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) - try { - val otherKubernetesResources = - resolvedDriverSpec.driverKubernetesResources ++ Seq(configMap) - addDriverOwnerReference(createdDriverPod, otherKubernetesResources) - kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() - } catch { - case NonFatal(e) => - kubernetesClient.pods().delete(createdDriverPod) - throw e - } - val sId = Seq(conf.namespace, driverPodName).mkString(":") - watcher.watchOrStop(sId) + var watch: Watch = null + val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) + try { + val otherKubernetesResources = resolvedDriverSpec.driverKubernetesResources ++ Seq(configMap) + addDriverOwnerReference(createdDriverPod, otherKubernetesResources) + kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() + } catch { + case NonFatal(e) => + kubernetesClient.pods().delete(createdDriverPod) + throw e + } + val sId = Seq(conf.namespace, driverPodName).mkString(":") + breakable { + while (true) { + val podWithName = kubernetesClient + .pods() + .withName(driverPodName) + // Reset resource to old before we start the watch, this is important for race conditions + watcher.reset() + watch = podWithName.watch(watcher) + + // Send the latest pod state we know to the watcher to make sure we didn't miss anything + watcher.eventReceived(Action.MODIFIED, podWithName.get()) + + // Break the while loop if the pod is completed or we don't want to wait + if(watcher.watchOrStop(sId)) { + watch.close() + break + } + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index ce3c80c0f85b1..aa27a9ef508ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import java.net.HttpURLConnection.HTTP_GONE import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.KubernetesDriverConf @@ -26,7 +27,8 @@ import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.internal.Logging private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { - def watchOrStop(submissionId: String): Unit + def watchOrStop(submissionId: String): Boolean + def reset(): Unit } /** @@ -42,10 +44,16 @@ private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) private var podCompleted = false + private var resourceTooOldReceived = false + private var pod = Option.empty[Pod] private def phase: String = pod.map(_.getStatus.getPhase).getOrElse("unknown") + override def reset(): Unit = { + resourceTooOldReceived = false + } + override def eventReceived(action: Action, pod: Pod): Unit = { this.pod = Option(pod) action match { @@ -62,7 +70,12 @@ private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) override def onClose(e: KubernetesClientException): Unit = { logDebug(s"Stopping watching application $appId with last-observed phase $phase") - closeWatch() + if(e != null && e.getCode == HTTP_GONE) { + resourceTooOldReceived = true + logDebug(s"Got HTTP Gone code, resource version changed in k8s api: $e") + } else { + closeWatch() + } } private def logLongStatus(): Unit = { @@ -78,20 +91,26 @@ private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) this.notifyAll() } - override def watchOrStop(sId: String): Unit = if (conf.get(WAIT_FOR_APP_COMPLETION)) { + override def watchOrStop(sId: String): Boolean = if (conf.get(WAIT_FOR_APP_COMPLETION)) { logInfo(s"Waiting for application ${conf.appName} with submission ID $sId to finish...") val interval = conf.get(REPORT_INTERVAL) synchronized { - while (!podCompleted) { + while (!podCompleted && !resourceTooOldReceived) { wait(interval) logInfo(s"Application status for $appId (phase: $phase)") } } - logInfo( - pod.map { p => s"Container final statuses:\n\n${containersDescription(p)}" } - .getOrElse("No containers were found in the driver pod.")) - logInfo(s"Application ${conf.appName} with submission ID $sId finished") + + if(podCompleted) { + logInfo( + pod.map { p => s"Container final statuses:\n\n${containersDescription(p)}" } + .getOrElse("No containers were found in the driver pod.")) + logInfo(s"Application ${conf.appName} with submission ID $sId finished") + } + podCompleted } else { logInfo(s"Deployed Spark application ${conf.appName} with submission ID $sId into Kubernetes") + // Always act like the application has completed since we don't want to wait for app completion + true } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala index fd8f6979c9e65..da7fe7cdda328 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala @@ -62,7 +62,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( .withoutLabel(SPARK_EXECUTOR_INACTIVE_LABEL, "true") .list() .getItems - .asScala) + .asScala.toSeq) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index d68dc3ebef5d8..5c192c690eba5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -131,7 +131,7 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul try { val snapshots = new ArrayList[ExecutorPodsSnapshot]() snapshotsBuffer.drainTo(snapshots) - onNewSnapshots(snapshots.asScala) + onNewSnapshots(snapshots.asScala.toSeq) } catch { case NonFatal(e) => logWarning("Exception when notifying snapshot subscriber.", e) } finally { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 5d49ac0bbaafa..d9ec3feb526ee 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -136,6 +136,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { createdResourcesArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) when(podOperations.create(FULL_EXPECTED_POD)).thenReturn(POD_WITH_OWNER_REFERENCE) when(namedPods.watch(loggingPodStatusWatcher)).thenReturn(mock[Watch]) + when(loggingPodStatusWatcher.watchOrStop(kconf.namespace + ":" + POD_NAME)).thenReturn(true) doReturn(resourceList) .when(kubernetesClient) .resourceList(createdResourcesArgumentCaptor.capture()) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala index 9ac7e0222054a..6dc052a5e6894 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala @@ -35,7 +35,7 @@ class DeterministicExecutorPodsSnapshotsStore extends ExecutorPodsSnapshotsStore override def stop(): Unit = {} override def notifySubscribers(): Unit = { - subscribers.foreach(_(snapshotsBuffer)) + subscribers.foreach(_(snapshotsBuffer.toSeq)) snapshotsBuffer.clear() } diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index d1e00cc0b5b10..4a55ead38aae2 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -27,11 +27,8 @@ spark-kubernetes-integration-tests_2.12 1.3.0 - 1.4.0 4.9.2 - 3.2.2 - 1.0 kubernetes-integration-tests diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala index becf9415c7506..fd67a03160afb 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.deploy.k8s.integrationtest +import org.apache.spark.internal.config import org.apache.spark.internal.config.Worker private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => @@ -28,18 +29,28 @@ private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => .set(Worker.WORKER_DECOMMISSION_ENABLED.key, "true") .set("spark.kubernetes.pyspark.pythonVersion", "3") .set("spark.kubernetes.container.image", pyImage) + .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true") + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true") + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true") + // Ensure we have somewhere to migrate our data too + .set("spark.executor.instances", "3") + // The default of 30 seconds is fine, but for testing we just want to get this done fast. + .set("spark.storage.decommission.replicationReattemptInterval", "1") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_DECOMISSIONING, mainClass = "", expectedLogOnCompletion = Seq( "Finished waiting, stopping Spark", - "decommissioning executor"), + "decommissioning executor", + "Final accumulator value is: 100"), appArgs = Array.empty[String], driverPodChecker = doBasicDriverPyPodCheck, executorPodChecker = doBasicExecutorPyPodCheck, appLocator = appLocator, isJVM = false, + pyFiles = None, + executorPatience = None, decommissioningTest = true) } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 65a2f1ff79697..279386d94b35d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -28,9 +28,10 @@ import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, Tag} -import org.scalatest.Matchers import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.concurrent.PatienceConfiguration.{Interval, Timeout} +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.{Minutes, Seconds, Span} import org.apache.spark.SparkFunSuite @@ -42,7 +43,8 @@ import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite - with DepsTestsSuite with DecommissionSuite with RTestsSuite with Logging with Eventually + // TODO(SPARK-32354): Fix and re-enable the R tests. + with DepsTestsSuite with DecommissionSuite /* with RTestsSuite */ with Logging with Eventually with Matchers { @@ -325,21 +327,36 @@ class KubernetesSuite extends SparkFunSuite val result = checkPodReady(namespace, name) result shouldBe (true) } - // Look for the string that indicates we're good to clean up - // on the driver + // Look for the string that indicates we're good to trigger decom on the driver logDebug("Waiting for first collect...") Eventually.eventually(TIMEOUT, INTERVAL) { assert(kubernetesTestComponents.kubernetesClient .pods() .withName(driverPodName) .getLog - .contains("Waiting to give nodes time to finish."), + .contains("Waiting to give nodes time to finish migration, decom exec 1."), "Decommission test did not complete first collect.") } // Delete the pod to simulate cluster scale down/migration. - val pod = kubernetesTestComponents.kubernetesClient.pods().withName(name) + // This will allow the pod to remain up for the grace period + val pod = kubernetesTestComponents.kubernetesClient.pods() + .withName(name) pod.delete() logDebug(s"Triggered pod decom/delete: $name deleted") + // Look for the string that indicates we should force kill the first + // Executor. This simulates the pod being fully lost. + logDebug("Waiting for second collect...") + Eventually.eventually(TIMEOUT, INTERVAL) { + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPodName) + .getLog + .contains("Waiting some more, please kill exec 1."), + "Decommission test did not complete second collect.") + } + logDebug("Force deleting") + val podNoGrace = pod.withGracePeriod(0) + podNoGrace.delete() } case Action.DELETED | Action.ERROR => execPods.remove(name) @@ -365,9 +382,10 @@ class KubernetesSuite extends SparkFunSuite .get(0) driverPodChecker(driverPod) - // If we're testing decommissioning we delete all the executors, but we should have - // an executor at some point. - Eventually.eventually(patienceTimeout, patienceInterval) { + + // If we're testing decommissioning we an executors, but we should have an executor + // at some point. + Eventually.eventually(TIMEOUT, patienceInterval) { execPods.values.nonEmpty should be (true) } execWatcher.close() @@ -482,6 +500,6 @@ private[spark] object KubernetesSuite { val SPARK_DFS_READ_WRITE_TEST = "org.apache.spark.examples.DFSReadWriteTest" val SPARK_REMOTE_MAIN_CLASS: String = "org.apache.spark.examples.SparkRemoteFileTest" val SPARK_DRIVER_MAIN_CLASS: String = "org.apache.spark.examples.DriverSubmissionTest" - val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + val TIMEOUT = PatienceConfiguration.Timeout(Span(3, Minutes)) val INTERVAL = PatienceConfiguration.Interval(Span(1, Seconds)) } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 4cfda8ab9ba77..af980f0494369 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -69,7 +69,7 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .set("spark.master", s"k8s://${kubernetesClient.getMasterUrl}") .set("spark.kubernetes.namespace", namespace) .set("spark.executor.cores", "1") - .set("spark.executors.instances", "1") + .set("spark.executor.instances", "1") .set("spark.app.name", "spark-test-app") .set(IS_TESTING.key, "false") .set(UI_ENABLED.key, "true") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index 7d76a22e6d363..cce842ce62f01 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -47,6 +47,6 @@ object ProcessUtils extends Logging { assert(proc.exitValue == 0, s"Failed to execute ${fullCommand.mkString(" ")}" + s"${if (dumpErrors) "\n" + outputLines.mkString("\n")}") - outputLines + outputLines.toSeq } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala index 3a6e5e7a6255d..b16ccb429074f 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala @@ -35,25 +35,6 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => isJVM = false) } - test("Run PySpark with Python2 to test a pyfiles example", k8sTestTag) { - sparkAppConf - .set("spark.kubernetes.container.image", pyImage) - .set("spark.kubernetes.pyspark.pythonVersion", "2") - runSparkApplicationAndVerifyCompletion( - appResource = PYSPARK_FILES, - mainClass = "", - expectedLogOnCompletion = Seq( - "Python runtime version check is: True", - "Python environment version check is: True", - "Python runtime version check for executor is: True"), - appArgs = Array("python"), - driverPodChecker = doBasicDriverPyPodCheck, - executorPodChecker = doBasicExecutorPyPodCheck, - appLocator = appLocator, - isJVM = false, - pyFiles = Some(PYSPARK_CONTAINER_TESTS)) - } - test("Run PySpark with Python3 to test a pyfiles example", k8sTestTag) { sparkAppConf .set("spark.kubernetes.container.image", pyImage) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala index 54a9dbf07816c..39ae94b17c63b 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala @@ -22,8 +22,8 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{Pod, SecretBuilder} import org.apache.commons.codec.binary.Base64 -import org.apache.commons.io.output.ByteArrayOutputStream import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite._ diff --git a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py index f68f24d49763d..d34e61611461c 100644 --- a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py +++ b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py @@ -31,14 +31,29 @@ .appName("PyMemoryTest") \ .getOrCreate() sc = spark._sc - rdd = sc.parallelize(range(10)) - rdd.collect() - print("Waiting to give nodes time to finish.") - time.sleep(5) + acc = sc.accumulator(0) + + def addToAcc(x): + acc.add(1) + return x + + initialRdd = sc.parallelize(range(100), 5) + accRdd = initialRdd.map(addToAcc) + # Trigger a shuffle so there are shuffle blocks to migrate + rdd = accRdd.map(lambda x: (x, x)).groupByKey() rdd.collect() - print("Waiting some more....") - time.sleep(10) + print("1st accumulator value is: " + str(acc.value)) + print("Waiting to give nodes time to finish migration, decom exec 1.") + print("...") + time.sleep(30) + rdd.count() + print("Waiting some more, please kill exec 1.") + print("...") + time.sleep(30) + print("Executor node should be deleted now") + rdd.count() rdd.collect() + print("Final accumulator value is: " + str(acc.value)) print("Finished waiting, stopping Spark.") spark.stop() print("Done, exiting Python") diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 26939ef23eaab..e5c18539a01ae 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -383,13 +383,13 @@ private[spark] class MesosClusterScheduler( taskId.split(s"${RETRY_SEP}").head } - private def adjust[A, B](m: collection.Map[A, B], k: A, default: B)(f: B => B) = { + private def adjust[A, B](m: Map[A, B], k: A, default: B)(f: B => B) = { m.updated(k, f(m.getOrElse(k, default))) } private def getDriverEnvironment(desc: MesosDriverDescription): Environment = { // TODO(mgummelt): Don't do this here. This should be passed as a --conf - val commandEnv = adjust(desc.command.environment, "SPARK_SUBMIT_OPTS", "")( + val commandEnv = adjust(desc.command.environment.toMap, "SPARK_SUBMIT_OPTS", "")( v => s"$v -D${config.DRIVER_FRAMEWORK_ID.key}=${getDriverFrameworkID(desc)}" ) @@ -686,14 +686,14 @@ private[spark] class MesosClusterScheduler( } scheduleTasks( - copyBuffer(driversToRetry), + copyBuffer(driversToRetry).toSeq, removeFromPendingRetryDrivers, currentOffers, tasks) // Then we walk through the queued drivers and try to schedule them. scheduleTasks( - copyBuffer(queuedDrivers), + copyBuffer(queuedDrivers).toSeq, removeFromQueuedDrivers, currentOffers, tasks) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 5e7a29ac6d344..32cd50298bc6c 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -491,8 +491,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val tasks = new mutable.HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil) // offerID -> resources - val remainingResources = mutable.Map(offers.map(offer => - (offer.getId.getValue, offer.getResourcesList)): _*) + val remainingResources = mutable.Map[String, JList[Resource]]() + remainingResources ++= offers.map(offer => (offer.getId.getValue, offer.getResourcesList)) var launchTasks = true diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 5784ee314aa17..2be8835f77e36 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -379,7 +379,7 @@ trait MesosSchedulerUtils extends Logging { } else { v.split(',').toSet } - ) + ).toMap } catch { case NonFatal(e) => throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index bb37bbd2d8046..2433c16c34af6 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -146,7 +146,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val resources = taskInfo.getResourcesList assert(scheduler.getResource(resources, "cpus") == 1.5) assert(scheduler.getResource(resources, "mem") == 1200) - val resourcesSeq: Seq[Resource] = resources.asScala + val resourcesSeq: Seq[Resource] = resources.asScala.toSeq val cpus = resourcesSeq.filter(_.getName == "cpus").toList assert(cpus.size == 2) assert(cpus.exists(_.getRole() == "role2")) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index 36ed84858dbfb..67ecf3242f52d 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -267,7 +267,8 @@ class MesosFineGrainedSchedulerBackendSuite properties = new Properties(), resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) + when(taskScheduler.resourceOffers( + expectedWorkerOffers.toIndexedSeq)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) @@ -379,7 +380,8 @@ class MesosFineGrainedSchedulerBackendSuite properties = new Properties(), resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) + when(taskScheduler.resourceOffers( + expectedWorkerOffers.toIndexedSeq)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(1) val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala index 0ed6fe66c56eb..f2efe86c3f9ec 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala @@ -25,7 +25,8 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import org.apache.mesos.Protos.{FrameworkInfo, Resource, Value} import org.mockito.Mockito._ -import org.scalatest._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} 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 7b121194d1b31..517a4af2e4b02 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 @@ -181,10 +181,12 @@ private[spark] class Client( // The app staging dir based on the STAGING_DIR configuration if configured // otherwise based on the users home directory. + // scalastyle:off FileSystemGet val appStagingBaseDir = sparkConf.get(STAGING_DIR) .map { new Path(_, UserGroupInformation.getCurrentUser.getShortUserName) } .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory()) stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId)) + // scalastyle:on FileSystemGet new CallerContext("CLIENT", sparkConf.get(APP_CALLER_CONTEXT), Option(appId.toString)).setCurrentContext() @@ -553,7 +555,7 @@ private[spark] class Client( } // Propagate the local URIs to the containers using the configuration. - sparkConf.set(SPARK_JARS, localJars) + sparkConf.set(SPARK_JARS, localJars.toSeq) case None => // No configuration, so fall back to uploading local jar files. @@ -628,7 +630,7 @@ private[spark] class Client( } } if (cachedSecondaryJarLinks.nonEmpty) { - sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks) + sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks.toSeq) } if (isClusterMode && args.primaryPyFile != null) { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index e6e0ea38ade94..e02fbd0c91495 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -91,11 +91,11 @@ private[spark] class ClientDistributedCacheManager() extends Logging { * Writes down information about cached files needed in executors to the given configuration. */ def updateConfiguration(conf: SparkConf): Unit = { - conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString)) - conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size)) - conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime)) - conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name())) - conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name())) + conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString).toSeq) + conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size).toSeq) + conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime).toSeq) + conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name()).toSeq) + conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name()).toSeq) } /** diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index cd0e7d5c87bc8..adbbbc01a0bd5 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -296,7 +296,7 @@ private[yarn] class YarnAllocator( val profResource = rpIdToYarnResource.get(id) val result = amClient.getMatchingRequests(getContainerPriority(id), location, profResource) .asScala.flatMap(_.asScala) - allContainerRequests(id) = result + allContainerRequests(id) = result.toSeq } allContainerRequests.toMap } @@ -308,7 +308,6 @@ private[yarn] class YarnAllocator( if (!rpIdToYarnResource.contains(rp.id)) { // Start with the application or default settings var heapMem = executorMemory.toLong - // Note we currently don't support off heap memory in ResourceProfile - SPARK-30794 var offHeapMem = executorOffHeapMemory.toLong var overheadMem = memoryOverhead.toLong var pysparkMem = pysparkWorkerMemory.toLong @@ -326,6 +325,8 @@ private[yarn] class YarnAllocator( overheadMem = execReq.amount case ResourceProfile.PYSPARK_MEM => pysparkMem = execReq.amount + case ResourceProfile.OFFHEAP_MEM => + offHeapMem = YarnSparkHadoopUtil.executorOffHeapMemorySizeAsMb(sparkConf, execReq) case ResourceProfile.CORES => cores = execReq.amount.toInt case "gpu" => @@ -426,13 +427,13 @@ private[yarn] class YarnAllocator( getNumExecutorsStarting, allocateResponse.getAvailableResources)) - handleAllocatedContainers(allocatedContainers.asScala) + handleAllocatedContainers(allocatedContainers.asScala.toSeq) } val completedContainers = allocateResponse.getCompletedContainersStatuses() if (completedContainers.size > 0) { logDebug("Completed %d containers".format(completedContainers.size)) - processCompletedContainers(completedContainers.asScala) + processCompletedContainers(completedContainers.asScala.toSeq) logDebug("Finished processing %d completed containers. Current running executor count: %d." .format(completedContainers.size, getNumExecutorsRunning)) } @@ -960,7 +961,7 @@ private[yarn] class YarnAllocator( } } - (localityMatched, localityUnMatched, localityFree) + (localityMatched.toSeq, localityUnMatched.toSeq, localityFree.toSeq) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 9d6b776a69d85..fe8990be7ee6f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils +import org.apache.spark.resource.ExecutorResourceRequest import org.apache.spark.util.Utils object YarnSparkHadoopUtil { @@ -187,11 +188,27 @@ object YarnSparkHadoopUtil { * Convert MEMORY_OFFHEAP_SIZE to MB Unit, return 0 if MEMORY_OFFHEAP_ENABLED is false. */ def executorOffHeapMemorySizeAsMb(sparkConf: SparkConf): Int = { + val sizeInMB = Utils.memoryStringToMb(sparkConf.get(MEMORY_OFFHEAP_SIZE).toString) + checkOffHeapEnabled(sparkConf, sizeInMB).toInt + } + + /** + * Get offHeap memory size from [[ExecutorResourceRequest]] + * return 0 if MEMORY_OFFHEAP_ENABLED is false. + */ + def executorOffHeapMemorySizeAsMb(sparkConf: SparkConf, + execRequest: ExecutorResourceRequest): Long = { + checkOffHeapEnabled(sparkConf, execRequest.amount) + } + + /** + * return 0 if MEMORY_OFFHEAP_ENABLED is false. + */ + def checkOffHeapEnabled(sparkConf: SparkConf, offHeapSize: Long): Long = { if (sparkConf.get(MEMORY_OFFHEAP_ENABLED)) { - val sizeInMB = Utils.memoryStringToMb(sparkConf.get(MEMORY_OFFHEAP_SIZE).toString) - require(sizeInMB > 0, + require(offHeapSize > 0, s"${MEMORY_OFFHEAP_SIZE.key} must be > 0 when ${MEMORY_OFFHEAP_ENABLED.key} == true") - sizeInMB + offHeapSize } else { 0 } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala index 669e39fb7c1c7..ce46ffa06f0fe 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala @@ -73,8 +73,8 @@ private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv, ResourceProfile) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env, resourceProfile) => new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, - arguments.resourcesFileOpt, resourceProfile) + arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath.toSeq, + env, arguments.resourcesFileOpt, resourceProfile) } val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index f8ef0d08d829c..20f5339c46fef 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -28,8 +28,9 @@ import scala.concurrent.duration._ import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.MiniYARNCluster -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.deploy.yarn.config._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 9c5c376ce5357..ea3acec3bb78b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -43,7 +43,8 @@ import org.apache.hadoop.yarn.util.Records import org.mockito.ArgumentMatchers.{any, anyBoolean, anyShort, eq => meq} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TestUtils} import org.apache.spark.deploy.yarn.ResourceRequestHelper._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala index d83a0d2efec1f..d64aad9ad470b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.deploy.yarn import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers -import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite import org.apache.spark.resource.ResourceProfile diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala index 4f77b9c99dd25..bfefdbc903b0c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/FailureTrackerSuite.scala @@ -16,7 +16,8 @@ */ package org.apache.spark.deploy.yarn -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.util.ManualClock diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala index f5ec531e26e0c..3e54dd0034307 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.deploy.yarn import org.apache.hadoop.yarn.api.records.Resource import org.apache.hadoop.yarn.util.Records -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.yarn.ResourceRequestHelper._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala index c07a4ac76b989..97615f5c936b0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala @@ -22,7 +22,8 @@ import java.util.Collections import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.mockito.Mockito._ -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.yarn.config.{YARN_EXCLUDE_NODES, YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 2003d0bb87b2d..63e2b97e0ecab 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -28,7 +28,9 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.ArgumentCaptor import org.mockito.Mockito._ -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index b7c9e83446012..14438bc141056 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -28,8 +28,9 @@ import scala.io.Source import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.api.python.PythonUtils diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 8c62069a8dd67..547bfca2891f1 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -23,7 +23,8 @@ import java.nio.charset.StandardCharsets import com.google.common.io.Files import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.deploy.yarn.config._ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index c88bb292aa77f..9cd37479a9db6 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -23,7 +23,8 @@ import java.nio.charset.StandardCharsets import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala index 632c66d77b707..63ac1af8a9127 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala @@ -21,7 +21,8 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.metrics2.MetricsRecordBuilder import org.mockito.ArgumentMatchers.{any, anyDouble, anyInt, anyLong} import org.mockito.Mockito.{mock, times, verify, when} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.network.server.OneForOneStreamManager diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 1a5a099217f55..46e596575533d 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -34,8 +34,10 @@ import org.apache.hadoop.service.ServiceStateException import org.apache.hadoop.yarn.api.records.ApplicationId import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext} -import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SecurityManager import org.apache.spark.SparkFunSuite diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 73ac14fdba1cf..c1dc57be56dff 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -264,6 +264,20 @@ This file is divided into 3 sections: of Commons Lang 2 (package org.apache.commons.lang.*) + + FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\) + + + extractOpt Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter 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 d29fa1319daa5..6fce7819897a6 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 @@ -16,7 +16,7 @@ grammar SqlBase; -@members { +@parser::members { /** * When false, INTERSECT is given the greater precedence over the other set * operations (UNION, EXCEPT and MINUS) as per the SQL standard. @@ -29,6 +29,13 @@ grammar SqlBase; */ public boolean legacy_exponent_literal_as_decimal_enabled = false; + /** + * When true, the behavior of keywords follows ANSI SQL standard. + */ + public boolean SQL_standard_keyword_behavior = false; +} + +@lexer::members { /** * Verify whether current token is a valid decimal token (which contains dot). * Returns true if the character that follows the token is not a digit or letter or underscore. @@ -51,11 +58,6 @@ grammar SqlBase; } } - /** - * When true, the behavior of keywords follows ANSI SQL standard. - */ - public boolean SQL_standard_keyword_behavior = false; - /** * This method will be called when we see '/*' and try to match it as a bracketed comment. * If the next character is '+', it should be parsed as hint later, and we cannot match @@ -229,6 +231,7 @@ statement comment=(STRING | NULL) #commentNamespace | COMMENT ON TABLE multipartIdentifier IS comment=(STRING | NULL) #commentTable | REFRESH TABLE multipartIdentifier #refreshTable + | REFRESH FUNCTION multipartIdentifier #refreshFunction | REFRESH (STRING | .*?) #refreshResource | CACHE LAZY? TABLE multipartIdentifier (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable @@ -240,11 +243,20 @@ statement | MSCK REPAIR TABLE multipartIdentifier #repairTable | op=(ADD | LIST) identifier (STRING | .*?) #manageResource | SET ROLE .*? #failNativeCommand + | SET TIME ZONE interval #setTimeZone + | SET TIME ZONE timezone=(STRING | LOCAL) #setTimeZone + | SET TIME ZONE .*? #setTimeZone + | SET configKey (EQ .*?)? #setQuotedConfiguration | SET .*? #setConfiguration - | RESET #resetConfiguration + | RESET configKey #resetQuotedConfiguration + | RESET .*? #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; +configKey + : quotedIdentifier + ; + unsupportedHiveNativeCommands : kw1=CREATE kw2=ROLE | kw1=DROP kw2=ROLE @@ -1190,6 +1202,7 @@ ansiNonReserved | VIEW | VIEWS | WINDOW + | ZONE //--ANSI-NON-RESERVED-END ; @@ -1221,6 +1234,7 @@ strictNonReserved ; nonReserved +//--DEFAULT-NON-RESERVED-START : ADD | AFTER | ALL @@ -1431,6 +1445,7 @@ nonReserved | TEMPORARY | TERMINATED | THEN + | TIME | TO | TOUCH | TRAILING @@ -1459,6 +1474,8 @@ nonReserved | WINDOW | WITH | YEAR + | ZONE +//--DEFAULT-NON-RESERVED-END ; // NOTE: If you add a new token in the list below, you should update the list of keywords @@ -1691,6 +1708,7 @@ TBLPROPERTIES: 'TBLPROPERTIES'; TEMPORARY: 'TEMPORARY' | 'TEMP'; TERMINATED: 'TERMINATED'; THEN: 'THEN'; +TIME: 'TIME'; TO: 'TO'; TOUCH: 'TOUCH'; TRAILING: 'TRAILING'; @@ -1721,6 +1739,7 @@ WHERE: 'WHERE'; WINDOW: 'WINDOW'; WITH: 'WITH'; YEAR: 'YEAR'; +ZONE: 'ZONE'; //--SPARK-KEYWORD-LIST-END //============================ // End of the keywords list diff --git a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 699b42bd6d60d..14b8f620017f6 100644 --- a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -39,8 +39,12 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def contains(k: String): Boolean = keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) - override def +[B1 >: T](kv: (String, B1)): Map[String, B1] = { - new CaseInsensitiveMap(originalMap + kv) + override def +[B1 >: T](kv: (String, B1)): CaseInsensitiveMap[B1] = { + new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(kv._1)) + kv) + } + + def ++(xs: TraversableOnce[(String, T)]): CaseInsensitiveMap[T] = { + xs.foldLeft(this)(_ + _) } override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator @@ -48,6 +52,8 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def -(key: String): Map[String, T] = { new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(key))) } + + def toMap: Map[String, T] = originalMap } object CaseInsensitiveMap { diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 1f8cc6c0f83a2..1db4b7a4f227b 100644 --- a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -39,8 +39,12 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def contains(k: String): Boolean = keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) - override def updated[B1 >: T](key: String, value: B1): Map[String, B1] = { - new CaseInsensitiveMap[B1](originalMap + (key -> value)) + override def updated[B1 >: T](key: String, value: B1): CaseInsensitiveMap[B1] = { + new CaseInsensitiveMap[B1](originalMap.filter(!_._1.equalsIgnoreCase(key)) + (key -> value)) + } + + def ++(xs: IterableOnce[(String, T)]): CaseInsensitiveMap[T] = { + xs.iterator.foldLeft(this) { (m, kv) => m.updated(kv._1, kv._2) } } override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator @@ -48,6 +52,8 @@ class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Ma override def removed(key: String): Map[String, T] = { new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(key))) } + + def toMap: Map[String, T] = originalMap } object CaseInsensitiveMap { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/OrderedFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/OrderedFilters.scala new file mode 100644 index 0000000000000..b7c8a0140ea66 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/OrderedFilters.scala @@ -0,0 +1,99 @@ +/* + * 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.catalyst + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType + +/** + * An instance of the class compiles filters to predicates and sorts them in + * the order which allows to apply the predicates to an internal row with partially + * initialized values, for instance converted from parsed CSV fields. + * + * @param filters The filters pushed down to a datasource. + * @param requiredSchema The schema with only fields requested by the upper layer. + */ +class OrderedFilters(filters: Seq[sources.Filter], requiredSchema: StructType) + extends StructFilters(filters, requiredSchema) { + /** + * Converted filters to predicates and grouped by maximum field index + * in the read schema. For example, if an filter refers to 2 attributes + * attrA with field index 5 and attrB with field index 10 in the read schema: + * 0 === $"attrA" or $"attrB" < 100 + * the filter is compiled to a predicate, and placed to the `predicates` + * array at the position 10. In this way, if there is a row with initialized + * fields from the 0 to 10 index, the predicate can be applied to the row + * to check that the row should be skipped or not. + * Multiple predicates with the same maximum reference index are combined + * by the `And` expression. + */ + private val predicates: Array[BasePredicate] = { + val len = requiredSchema.fields.length + val groupedPredicates = Array.fill[BasePredicate](len)(null) + val groupedFilters = Array.fill(len)(Seq.empty[sources.Filter]) + for (filter <- filters) { + val refs = filter.references + val index = if (refs.isEmpty) { + // For example, `AlwaysTrue` and `AlwaysFalse` doesn't have any references + // Filters w/o refs always return the same result. Taking into account + // that predicates are combined via `And`, we can apply such filters only + // once at the position 0. + 0 + } else { + // readSchema must contain attributes of all filters. + // Accordingly, `fieldIndex()` returns a valid index always. + refs.map(requiredSchema.fieldIndex).max + } + groupedFilters(index) :+= filter + } + if (len > 0 && groupedFilters(0).nonEmpty) { + // We assume that filters w/o refs like `AlwaysTrue` and `AlwaysFalse` + // can be evaluated faster that others. We put them in front of others. + val (literals, others) = groupedFilters(0).partition(_.references.isEmpty) + groupedFilters(0) = literals ++ others + } + for (i <- 0 until len) { + if (groupedFilters(i).nonEmpty) { + groupedPredicates(i) = toPredicate(groupedFilters(i)) + } + } + groupedPredicates + } + + /** + * Applies all filters that refer to row fields at the positions from 0 to `index`. + * @param row The internal row to check. + * @param index Maximum field index. The function assumes that all fields + * from 0 to `index` position are set. + * @return false` iff row fields at the position from 0 to `index` pass filters + * or there are no applicable filters + * otherwise `false` if at least one of the filters returns `false`. + */ + def skipRow(row: InternalRow, index: Int): Boolean = { + assert(0 <= index && index < requiredSchema.fields.length, + "Index is out of the valid range: it must point out to a field of the required schema.") + val predicate = predicates(index) + predicate != null && !predicate.eval(row) + } + + // The filters are applied sequentially, and no need to track which filter references + // point out to already set row values. The `reset()` method is trivial because + // the filters don't have any states. + def reset(): Unit = {} +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala new file mode 100644 index 0000000000000..fed1b323f5773 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala @@ -0,0 +1,166 @@ +/* + * 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.catalyst + +import scala.util.Try + +import org.apache.spark.sql.catalyst.StructFilters._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.{BooleanType, StructType} + +/** + * The class provides API for applying pushed down filters to partially or + * fully set internal rows that have the struct schema. + * + * `StructFilters` assumes that: + * - `reset()` is called before any `skipRow()` calls for new row. + * + * @param pushedFilters The pushed down source filters. The filters should refer to + * the fields of the provided schema. + * @param schema The required schema of records from datasource files. + */ +abstract class StructFilters(pushedFilters: Seq[sources.Filter], schema: StructType) { + + protected val filters = StructFilters.pushedFilters(pushedFilters.toArray, schema) + + /** + * Applies pushed down source filters to the given row assuming that + * value at `index` has been already set. + * + * @param row The row with fully or partially set values. + * @param index The index of already set value. + * @return `true` if currently processed row can be skipped otherwise false. + */ + def skipRow(row: InternalRow, index: Int): Boolean + + /** + * Resets states of pushed down filters. The method must be called before + * precessing any new row otherwise `skipRow()` may return wrong result. + */ + def reset(): Unit + + /** + * Compiles source filters to a predicate. + */ + def toPredicate(filters: Seq[sources.Filter]): BasePredicate = { + val reducedExpr = filters + .sortBy(_.references.length) + .flatMap(filterToExpression(_, toRef)) + .reduce(And) + Predicate.create(reducedExpr) + } + + // Finds a filter attribute in the schema and converts it to a `BoundReference` + private def toRef(attr: String): Option[BoundReference] = { + // The names have been normalized and case sensitivity is not a concern here. + schema.getFieldIndex(attr).map { index => + val field = schema(index) + BoundReference(index, field.dataType, field.nullable) + } + } +} + +object StructFilters { + private def checkFilterRefs(filter: sources.Filter, fieldNames: Set[String]): Boolean = { + // The names have been normalized and case sensitivity is not a concern here. + filter.references.forall(fieldNames.contains) + } + + /** + * Returns the filters currently supported by the datasource. + * @param filters The filters pushed down to the datasource. + * @param schema data schema of datasource files. + * @return a sub-set of `filters` that can be handled by the datasource. + */ + def pushedFilters(filters: Array[sources.Filter], schema: StructType): Array[sources.Filter] = { + val fieldNames = schema.fieldNames.toSet + filters.filter(checkFilterRefs(_, fieldNames)) + } + + private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = { + a.zip(b).headOption + } + + private def toLiteral(value: Any): Option[Literal] = { + Try(Literal(value)).toOption + } + + /** + * Converts a filter to an expression and binds it to row positions. + * + * @param filter The filter to convert. + * @param toRef The function converts a filter attribute to a bound reference. + * @return some expression with resolved attributes or `None` if the conversion + * of the given filter to an expression is impossible. + */ + def filterToExpression( + filter: sources.Filter, + toRef: String => Option[BoundReference]): Option[Expression] = { + def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { + zip(toRef(name), toLiteral(value)) + } + def translate(filter: sources.Filter): Option[Expression] = filter match { + case sources.And(left, right) => + zip(translate(left), translate(right)).map(And.tupled) + case sources.Or(left, right) => + zip(translate(left), translate(right)).map(Or.tupled) + case sources.Not(child) => + translate(child).map(Not) + case sources.EqualTo(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualTo.tupled) + case sources.EqualNullSafe(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualNullSafe.tupled) + case sources.IsNull(attribute) => + toRef(attribute).map(IsNull) + case sources.IsNotNull(attribute) => + toRef(attribute).map(IsNotNull) + case sources.In(attribute, values) => + val literals = values.toSeq.flatMap(toLiteral) + if (literals.length == values.length) { + toRef(attribute).map(In(_, literals)) + } else { + None + } + case sources.GreaterThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThan.tupled) + case sources.GreaterThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThanOrEqual.tupled) + case sources.LessThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThan.tupled) + case sources.LessThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThanOrEqual.tupled) + case sources.StringContains(attribute, value) => + zipAttributeAndValue(attribute, value).map(Contains.tupled) + case sources.StringStartsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(StartsWith.tupled) + case sources.StringEndsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(EndsWith.tupled) + case sources.AlwaysTrue() => + Some(Literal(true, BooleanType)) + case sources.AlwaysFalse() => + Some(Literal(false, BooleanType)) + } + translate(filter) + } +} + +class NoopFilters extends StructFilters(Seq.empty, new StructType()) { + override def skipRow(row: InternalRow, index: Int): Boolean = false + override def reset(): Unit = {} +} 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 023ef2ee17473..477863a1b86d0 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 @@ -200,6 +200,11 @@ class Analyzer( val postHocResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( + Batch("Substitution", fixedPoint, + CTESubstitution, + WindowsSubstitution, + EliminateUnions, + new SubstituteUnresolvedOrdinals(conf)), Batch("Disable Hints", Once, new ResolveHints.DisableHints(conf)), Batch("Hints", fixedPoint, @@ -207,11 +212,6 @@ class Analyzer( new ResolveHints.ResolveCoalesceHints(conf)), Batch("Simple Sanity Check", Once, LookupFunctions), - Batch("Substitution", fixedPoint, - CTESubstitution, - WindowsSubstitution, - EliminateUnions, - new SubstituteUnresolvedOrdinals(conf)), Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: ResolveNamespace(catalogManager) :: @@ -249,6 +249,7 @@ class Analyzer( ResolveTimeZone(conf) :: ResolveRandomSeed :: ResolveBinaryArithmetic :: + ResolveUnion :: TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), @@ -791,9 +792,9 @@ class Analyzer( // AggregateFunction's with the exception of First and Last in their default mode // (which we handle) and possibly some Hive UDAF's. case First(expr, _) => - First(ifExpr(expr), Literal(true)) + First(ifExpr(expr), true) case Last(expr, _) => - Last(ifExpr(expr), Literal(true)) + Last(ifExpr(expr), true) case a: AggregateFunction => a.withNewChildren(a.children.map(ifExpr)) }.transform { @@ -1192,11 +1193,24 @@ class Analyzer( if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy(projectList = newAliases(projectList)))) + // We don't need to search child plan recursively if the projectList of a Project + // is only composed of Alias and doesn't contain any conflicting attributes. + // Because, even if the child plan has some conflicting attributes, the attributes + // will be aliased to non-conflicting attributes by the Project at the end. + case _ @ Project(projectList, _) + if findAliases(projectList).size == projectList.size => + Nil + case oldVersion @ Aggregate(_, aggregateExpressions, _) if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy( aggregateExpressions = newAliases(aggregateExpressions)))) + // We don't search the child plan recursively for the same reason as the above Project. + case _ @ Aggregate(_, aggregateExpressions, _) + if findAliases(aggregateExpressions).size == aggregateExpressions.size => + Nil + case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) @@ -1237,20 +1251,50 @@ class Analyzer( if (conflictPlans.isEmpty) { right } else { - val attributeRewrites = AttributeMap(conflictPlans.flatMap { - case (oldRelation, newRelation) => oldRelation.output.zip(newRelation.output)}) - val conflictPlanMap = conflictPlans.toMap - // transformDown so that we can replace all the old Relations in one turn due to - // the reason that `conflictPlans` are also collected in pre-order. - right transformDown { - case r => conflictPlanMap.getOrElse(r, r) - } transformUp { - case other => other transformExpressions { + rewritePlan(right, conflictPlans.toMap)._1 + } + } + + private def rewritePlan(plan: LogicalPlan, conflictPlanMap: Map[LogicalPlan, LogicalPlan]) + : (LogicalPlan, Seq[(Attribute, Attribute)]) = { + if (conflictPlanMap.contains(plan)) { + // If the plan is the one that conflict the with left one, we'd + // just replace it with the new plan and collect the rewrite + // attributes for the parent node. + val newRelation = conflictPlanMap(plan) + newRelation -> plan.output.zip(newRelation.output) + } else { + val attrMapping = new mutable.ArrayBuffer[(Attribute, Attribute)]() + val newPlan = plan.mapChildren { child => + // If not, we'd rewrite child plan recursively until we find the + // conflict node or reach the leaf node. + val (newChild, childAttrMapping) = rewritePlan(child, conflictPlanMap) + attrMapping ++= childAttrMapping.filter { case (oldAttr, _) => + // `attrMapping` is not only used to replace the attributes of the current `plan`, + // but also to be propagated to the parent plans of the current `plan`. Therefore, + // the `oldAttr` must be part of either `plan.references` (so that it can be used to + // replace attributes of the current `plan`) or `plan.outputSet` (so that it can be + // used by those parent plans). + (plan.outputSet ++ plan.references).contains(oldAttr) + } + newChild + } + + if (attrMapping.isEmpty) { + newPlan -> attrMapping + } else { + assert(!attrMapping.groupBy(_._1.exprId) + .exists(_._2.map(_._2.exprId).distinct.length > 1), + "Found duplicate rewrite attributes") + val attributeRewrites = AttributeMap(attrMapping) + // Using attrMapping from the children plans to rewrite their parent node. + // Note that we shouldn't rewrite a node using attrMapping from its sibling nodes. + newPlan.transformExpressions { case a: Attribute => dedupAttr(a, attributeRewrites) case s: SubqueryExpression => s.withNewPlan(dedupOuterReferencesInSubquery(s.plan, attributeRewrites)) - } + } -> attrMapping } } } @@ -1387,10 +1431,11 @@ class Analyzer( i.copy(right = dedupRight(left, right)) case e @ Except(left, right, _) if !e.duplicateResolved => e.copy(right = dedupRight(left, right)) - case u @ Union(children) if !u.duplicateResolved => + // Only after we finish by-name resolution for Union + case u: Union if !u.byName && !u.duplicateResolved => // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing // feature in streaming. - val newChildren = children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => + val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => head +: tail.map { case child if head.outputSet.intersect(child.outputSet).isEmpty => child @@ -1886,11 +1931,17 @@ class Analyzer( } /** + * Replaces [[UnresolvedFunc]]s with concrete [[LogicalPlan]]s. * Replaces [[UnresolvedFunction]]s with concrete [[Expression]]s. */ object ResolveFunctions extends Rule[LogicalPlan] { val trimWarningEnabled = new AtomicBoolean(true) def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + // Resolve functions with concrete relations from v2 catalog. + case UnresolvedFunc(multipartIdent) => + val funcIdent = parseSessionCatalogFunctionIdentifier(multipartIdent) + ResolvedFunc(Identifier.of(funcIdent.database.toArray, funcIdent.funcName)) + case q: LogicalPlan => q transformExpressions { case u if !u.childrenResolved => u // Skip until children are resolved. @@ -1922,15 +1973,9 @@ class Analyzer( } // We get an aggregate function, we need to wrap it in an AggregateExpression. case agg: AggregateFunction => - // TODO: SPARK-30276 Support Filter expression allows simultaneous use of DISTINCT - if (filter.isDefined) { - if (isDistinct) { - failAnalysis("DISTINCT and FILTER cannot be used in aggregate functions " + - "at the same time") - } else if (!filter.get.deterministic) { - failAnalysis("FILTER expression is non-deterministic, " + - "it cannot be used in aggregate functions") - } + if (filter.isDefined && !filter.get.deterministic) { + failAnalysis("FILTER expression is non-deterministic, " + + "it cannot be used in aggregate functions") } AggregateExpression(agg, Complete, isDistinct, filter) // This function is not an aggregate function, just return the resolved one. @@ -3398,7 +3443,7 @@ object EliminateSubqueryAliases extends Rule[LogicalPlan] { */ object EliminateUnions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case Union(children) if children.size == 1 => children.head + case u: Union if u.children.size == 1 => u.children.head } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 3989df5d29467..7e73667e4b85f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -354,6 +354,7 @@ object FunctionRegistry { expression[StringLocate]("position", true), expression[FormatString]("printf", true), expression[RegExpExtract]("regexp_extract"), + expression[RegExpExtractAll]("regexp_extract_all"), expression[RegExpReplace]("regexp_replace"), expression[StringRepeat]("repeat"), expression[StringReplace]("replace"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala new file mode 100644 index 0000000000000..693a5a4e75443 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala @@ -0,0 +1,100 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} +import org.apache.spark.sql.catalyst.optimizer.CombineUnions +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.SchemaUtils + +/** + * Resolves different children of Union to a common set of columns. + */ +object ResolveUnion extends Rule[LogicalPlan] { + private def unionTwoSides( + left: LogicalPlan, + right: LogicalPlan, + allowMissingCol: Boolean): LogicalPlan = { + val resolver = SQLConf.get.resolver + val leftOutputAttrs = left.output + val rightOutputAttrs = right.output + + // Builds a project list for `right` based on `left` output names + val rightProjectList = leftOutputAttrs.map { lattr => + rightOutputAttrs.find { rattr => resolver(lattr.name, rattr.name) }.getOrElse { + if (allowMissingCol) { + Alias(Literal(null, lattr.dataType), lattr.name)() + } else { + throw new AnalysisException( + s"""Cannot resolve column name "${lattr.name}" among """ + + s"""(${rightOutputAttrs.map(_.name).mkString(", ")})""") + } + } + } + + // Delegates failure checks to `CheckAnalysis` + val notFoundAttrs = rightOutputAttrs.diff(rightProjectList) + val rightChild = Project(rightProjectList ++ notFoundAttrs, right) + + // Builds a project for `logicalPlan` based on `right` output names, if allowing + // missing columns. + val leftChild = if (allowMissingCol) { + val missingAttrs = notFoundAttrs.map { attr => + Alias(Literal(null, attr.dataType), attr.name)() + } + if (missingAttrs.nonEmpty) { + Project(leftOutputAttrs ++ missingAttrs, left) + } else { + left + } + } else { + left + } + Union(leftChild, rightChild) + } + + // Check column name duplication + private def checkColumnNames(left: LogicalPlan, right: LogicalPlan): Unit = { + val caseSensitiveAnalysis = SQLConf.get.caseSensitiveAnalysis + val leftOutputAttrs = left.output + val rightOutputAttrs = right.output + + SchemaUtils.checkColumnNameDuplication( + leftOutputAttrs.map(_.name), + "in the left attributes", + caseSensitiveAnalysis) + SchemaUtils.checkColumnNameDuplication( + rightOutputAttrs.map(_.name), + "in the right attributes", + caseSensitiveAnalysis) + } + + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { + case e if !e.childrenResolved => e + + case Union(children, byName, allowMissingCol) if byName => + val union = children.reduceLeft { (left, right) => + checkColumnNames(left, right) + unionTwoSides(left, right, allowMissingCol) + } + CombineUnions(union) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 3484108a5503f..604a082be4e55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -341,10 +341,10 @@ object TypeCoercion { assert(newChildren.length == 2) Intersect(newChildren.head, newChildren.last, isAll) - case s: Union if s.childrenResolved && + case s: Union if s.childrenResolved && !s.byName && s.children.forall(_.output.length == s.children.head.output.length) && !s.resolved => val newChildren: Seq[LogicalPlan] = buildNewChildrenWithWiderTypes(s.children) - s.makeCopy(Array(newChildren)) + s.copy(children = newChildren) } /** Build new children with the widest types for each attribute among all the children */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index f3d40c6d36cc3..a16763f2cf943 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.catalog.CatalogFunction import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsNamespaces, Table, TableCatalog} @@ -50,6 +51,15 @@ case class UnresolvedTableOrView(multipartIdentifier: Seq[String]) extends LeafN override def output: Seq[Attribute] = Nil } +/** + * Holds the name of a function that has yet to be looked up in a catalog. It will be resolved to + * [[ResolvedFunc]] during analysis. + */ +case class UnresolvedFunc(multipartIdentifier: Seq[String]) extends LeafNode { + override lazy val resolved: Boolean = false + override def output: Seq[Attribute] = Nil +} + /** * A plan containing resolved namespace. */ @@ -74,3 +84,13 @@ case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: T case class ResolvedView(identifier: Identifier) extends LeafNode { override def output: Seq[Attribute] = Nil } + +/** + * A plan containing resolved function. + */ +// TODO: create a generic representation for v1, v2 function, after we add function +// support to v2 catalog. For now we only need the identifier to fallback to v1 command. +case class ResolvedFunc(identifier: Identifier) + extends LeafNode { + override def output: Seq[Attribute] = Nil +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index b79857cdccd22..6fba3156c3919 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI import java.util.Locale import java.util.concurrent.Callable +import java.util.concurrent.TimeUnit import javax.annotation.concurrent.GuardedBy import scala.collection.mutable @@ -135,7 +136,16 @@ class SessionCatalog( private val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = { val cacheSize = conf.tableRelationCacheSize - CacheBuilder.newBuilder().maximumSize(cacheSize).build[QualifiedTableName, LogicalPlan]() + val cacheTTL = conf.metadataCacheTTL + + var builder = CacheBuilder.newBuilder() + .maximumSize(cacheSize) + + if (cacheTTL > 0) { + builder = builder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS) + } + + builder.build[QualifiedTableName, LogicalPlan]() } /** This method provides a way to get a cached plan. */ @@ -209,12 +219,20 @@ class SessionCatalog( "you cannot create a database with this name.") } validateName(dbName) - val qualifiedPath = makeQualifiedPath(dbDefinition.locationUri) externalCatalog.createDatabase( - dbDefinition.copy(name = dbName, locationUri = qualifiedPath), + dbDefinition.copy(name = dbName, locationUri = makeQualifiedDBPath(dbDefinition.locationUri)), ignoreIfExists) } + private def makeQualifiedDBPath(locationUri: URI): URI = { + if (locationUri.isAbsolute) { + locationUri + } else { + val fullPath = new Path(conf.warehousePath, CatalogUtils.URIToString(locationUri)) + makeQualifiedPath(fullPath.toUri) + } + } + def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { val dbName = formatDatabaseName(db) if (dbName == DEFAULT_DATABASE) { @@ -231,7 +249,8 @@ class SessionCatalog( def alterDatabase(dbDefinition: CatalogDatabase): Unit = { val dbName = formatDatabaseName(dbDefinition.name) requireDbExists(dbName) - externalCatalog.alterDatabase(dbDefinition.copy(name = dbName)) + externalCatalog.alterDatabase(dbDefinition.copy( + name = dbName, locationUri = makeQualifiedDBPath(dbDefinition.locationUri))) } def getDatabaseMetadata(db: String): CatalogDatabase = { @@ -273,8 +292,7 @@ class SessionCatalog( * by users. */ def getDefaultDBPath(db: String): URI = { - val database = formatDatabaseName(db) - new Path(new Path(conf.warehousePath), database + ".db").toUri + CatalogUtils.stringToURI(formatDatabaseName(db) + ".db") } // ---------------------------------------------------------------------------- @@ -307,7 +325,7 @@ class SessionCatalog( && !tableDefinition.storage.locationUri.get.isAbsolute) { // make the location of the table qualified. val qualifiedTableLocation = - makeQualifiedPath(tableDefinition.storage.locationUri.get) + makeQualifiedTablePath(tableDefinition.storage.locationUri.get, db) tableDefinition.copy( storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), identifier = tableIdentifier) @@ -340,6 +358,16 @@ class SessionCatalog( } } + private def makeQualifiedTablePath(locationUri: URI, database: String): URI = { + if (locationUri.isAbsolute) { + locationUri + } else { + val dbName = formatDatabaseName(database) + val dbLocation = makeQualifiedDBPath(getDatabaseMetadata(dbName).locationUri) + new Path(new Path(dbLocation), CatalogUtils.URIToString(locationUri)).toUri + } + } + /** * Alter the metadata of an existing metastore table identified by `tableDefinition`. * @@ -359,7 +387,7 @@ class SessionCatalog( && !tableDefinition.storage.locationUri.get.isAbsolute) { // make the location of the table qualified. val qualifiedTableLocation = - makeQualifiedPath(tableDefinition.storage.locationUri.get) + makeQualifiedTablePath(tableDefinition.storage.locationUri.get, db) tableDefinition.copy( storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), identifier = tableIdentifier) @@ -1341,6 +1369,14 @@ class SessionCatalog( functionRegistry.registerFunction(func, info, builder) } + /** + * Unregister a temporary or permanent function from a session-specific [[FunctionRegistry]] + * Return true if function exists. + */ + def unregisterFunction(name: FunctionIdentifier): Boolean = { + functionRegistry.dropFunction(name) + } + /** * Drop a temporary function. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala deleted file mode 100644 index b50a76a496556..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.csv - -import scala.util.Try - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources -import org.apache.spark.sql.types.{BooleanType, StructType} - -/** - * An instance of the class compiles filters to predicates and allows to - * apply the predicates to an internal row with partially initialized values - * converted from parsed CSV fields. - * - * @param filters The filters pushed down to CSV datasource. - * @param requiredSchema The schema with only fields requested by the upper layer. - */ -class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) { - /** - * Converted filters to predicates and grouped by maximum field index - * in the read schema. For example, if an filter refers to 2 attributes - * attrA with field index 5 and attrB with field index 10 in the read schema: - * 0 === $"attrA" or $"attrB" < 100 - * the filter is compiled to a predicate, and placed to the `predicates` - * array at the position 10. In this way, if there is a row with initialized - * fields from the 0 to 10 index, the predicate can be applied to the row - * to check that the row should be skipped or not. - * Multiple predicates with the same maximum reference index are combined - * by the `And` expression. - */ - private val predicates: Array[BasePredicate] = { - val len = requiredSchema.fields.length - val groupedPredicates = Array.fill[BasePredicate](len)(null) - if (SQLConf.get.csvFilterPushDown) { - val groupedFilters = Array.fill(len)(Seq.empty[sources.Filter]) - for (filter <- filters) { - val refs = filter.references - val index = if (refs.isEmpty) { - // For example, AlwaysTrue and AlwaysFalse doesn't have any references - // Filters w/o refs always return the same result. Taking into account - // that predicates are combined via And, we can apply such filters only - // once at the position 0. - 0 - } else { - // readSchema must contain attributes of all filters. - // Accordingly, fieldIndex() returns a valid index always. - refs.map(requiredSchema.fieldIndex).max - } - groupedFilters(index) :+= filter - } - if (len > 0 && !groupedFilters(0).isEmpty) { - // We assume that filters w/o refs like AlwaysTrue and AlwaysFalse - // can be evaluated faster that others. We put them in front of others. - val (literals, others) = groupedFilters(0).partition(_.references.isEmpty) - groupedFilters(0) = literals ++ others - } - for (i <- 0 until len) { - if (!groupedFilters(i).isEmpty) { - val reducedExpr = groupedFilters(i) - .flatMap(CSVFilters.filterToExpression(_, toRef)) - .reduce(And) - groupedPredicates(i) = Predicate.create(reducedExpr) - } - } - } - groupedPredicates - } - - /** - * Applies all filters that refer to row fields at the positions from 0 to index. - * @param row The internal row to check. - * @param index Maximum field index. The function assumes that all fields - * from 0 to index position are set. - * @return false iff row fields at the position from 0 to index pass filters - * or there are no applicable filters - * otherwise false if at least one of the filters returns false. - */ - def skipRow(row: InternalRow, index: Int): Boolean = { - val predicate = predicates(index) - predicate != null && !predicate.eval(row) - } - - // Finds a filter attribute in the read schema and converts it to a `BoundReference` - private def toRef(attr: String): Option[BoundReference] = { - requiredSchema.getFieldIndex(attr).map { index => - val field = requiredSchema(index) - BoundReference(requiredSchema.fieldIndex(attr), field.dataType, field.nullable) - } - } -} - -object CSVFilters { - private def checkFilterRefs(filter: sources.Filter, schema: StructType): Boolean = { - val fieldNames = schema.fields.map(_.name).toSet - filter.references.forall(fieldNames.contains(_)) - } - - /** - * Returns the filters currently supported by CSV datasource. - * @param filters The filters pushed down to CSV datasource. - * @param schema data schema of CSV files. - * @return a sub-set of `filters` that can be handled by CSV datasource. - */ - def pushedFilters(filters: Array[sources.Filter], schema: StructType): Array[sources.Filter] = { - filters.filter(checkFilterRefs(_, schema)) - } - - private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = { - a.zip(b).headOption - } - - private def toLiteral(value: Any): Option[Literal] = { - Try(Literal(value)).toOption - } - - /** - * Converts a filter to an expression and binds it to row positions. - * - * @param filter The filter to convert. - * @param toRef The function converts a filter attribute to a bound reference. - * @return some expression with resolved attributes or None if the conversion - * of the given filter to an expression is impossible. - */ - def filterToExpression( - filter: sources.Filter, - toRef: String => Option[BoundReference]): Option[Expression] = { - def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { - zip(toRef(name), toLiteral(value)) - } - def translate(filter: sources.Filter): Option[Expression] = filter match { - case sources.And(left, right) => - zip(translate(left), translate(right)).map(And.tupled) - case sources.Or(left, right) => - zip(translate(left), translate(right)).map(Or.tupled) - case sources.Not(child) => - translate(child).map(Not) - case sources.EqualTo(attribute, value) => - zipAttributeAndValue(attribute, value).map(EqualTo.tupled) - case sources.EqualNullSafe(attribute, value) => - zipAttributeAndValue(attribute, value).map(EqualNullSafe.tupled) - case sources.IsNull(attribute) => - toRef(attribute).map(IsNull) - case sources.IsNotNull(attribute) => - toRef(attribute).map(IsNotNull) - case sources.In(attribute, values) => - val literals = values.toSeq.flatMap(toLiteral) - if (literals.length == values.length) { - toRef(attribute).map(In(_, literals)) - } else { - None - } - case sources.GreaterThan(attribute, value) => - zipAttributeAndValue(attribute, value).map(GreaterThan.tupled) - case sources.GreaterThanOrEqual(attribute, value) => - zipAttributeAndValue(attribute, value).map(GreaterThanOrEqual.tupled) - case sources.LessThan(attribute, value) => - zipAttributeAndValue(attribute, value).map(LessThan.tupled) - case sources.LessThanOrEqual(attribute, value) => - zipAttributeAndValue(attribute, value).map(LessThanOrEqual.tupled) - case sources.StringContains(attribute, value) => - zipAttributeAndValue(attribute, value).map(Contains.tupled) - case sources.StringStartsWith(attribute, value) => - zipAttributeAndValue(attribute, value).map(StartsWith.tupled) - case sources.StringEndsWith(attribute, value) => - zipAttributeAndValue(attribute, value).map(EndsWith.tupled) - case sources.AlwaysTrue() => - Some(Literal(true, BooleanType)) - case sources.AlwaysFalse() => - Some(Literal(false, BooleanType)) - } - translate(filter) - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 898b963fd0ab5..b5c14a193ddee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -25,10 +25,11 @@ import com.univocity.parsers.csv.CsvParser import org.apache.spark.SparkUpgradeException import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters} import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -98,7 +99,11 @@ class UnivocityParser( legacyFormat = FAST_DATE_FORMAT, isParsing = true) - private val csvFilters = new CSVFilters(filters, requiredSchema) + private val csvFilters = if (SQLConf.get.csvFilterPushDown) { + new OrderedFilters(filters, requiredSchema) + } else { + new NoopFilters + } // Retrieve the raw record string. private def getCurrentInput: UTF8String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index e27c021556377..312c78a2ef1e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -297,6 +297,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) + private val legacyCastToStr = SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) + // The brackets that are used in casting structs and maps to strings + private val (leftBracket, rightBracket) = if (legacyCastToStr) ("[", "]") else ("{", "}") + // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { case CalendarIntervalType => @@ -317,7 +321,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit var i = 1 while (i < array.numElements) { builder.append(",") - if (!array.isNullAt(i)) { + if (array.isNullAt(i)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(toUTF8String(array.get(i, et)).asInstanceOf[UTF8String]) } @@ -330,7 +336,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case MapType(kt, vt, _) => buildCast[MapData](_, map => { val builder = new UTF8StringBuilder - builder.append("[") + builder.append(leftBracket) if (map.numElements > 0) { val keyArray = map.keyArray() val valueArray = map.valueArray() @@ -338,7 +344,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val valueToUTF8String = castToString(vt) builder.append(keyToUTF8String(keyArray.get(0, kt)).asInstanceOf[UTF8String]) builder.append(" ->") - if (!valueArray.isNullAt(0)) { + if (valueArray.isNullAt(0)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(valueToUTF8String(valueArray.get(0, vt)).asInstanceOf[UTF8String]) } @@ -347,7 +355,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit builder.append(", ") builder.append(keyToUTF8String(keyArray.get(i, kt)).asInstanceOf[UTF8String]) builder.append(" ->") - if (!valueArray.isNullAt(i)) { + if (valueArray.isNullAt(i)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(valueToUTF8String(valueArray.get(i, vt)) .asInstanceOf[UTF8String]) @@ -355,30 +365,34 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit i += 1 } } - builder.append("]") + builder.append(rightBracket) builder.build() }) case StructType(fields) => buildCast[InternalRow](_, row => { val builder = new UTF8StringBuilder - builder.append("[") + builder.append(leftBracket) if (row.numFields > 0) { val st = fields.map(_.dataType) val toUTF8StringFuncs = st.map(castToString) - if (!row.isNullAt(0)) { + if (row.isNullAt(0)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(toUTF8StringFuncs(0)(row.get(0, st(0))).asInstanceOf[UTF8String]) } var i = 1 while (i < row.numFields) { builder.append(",") - if (!row.isNullAt(i)) { + if (row.isNullAt(i)) { + if (!legacyCastToStr) builder.append(" null") + } else { builder.append(" ") builder.append(toUTF8StringFuncs(i)(row.get(i, st(i))).asInstanceOf[UTF8String]) } i += 1 } } - builder.append("]") + builder.append(rightBracket) builder.build() }) case pudt: PythonUserDefinedType => castToString(pudt.sqlType) @@ -891,6 +905,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit """ } + private def outNullElem(buffer: ExprValue): Block = { + if (legacyCastToStr) code"" else code"""$buffer.append(" null");""" + } + private def writeArrayToStringBuilder( et: DataType, array: ExprValue, @@ -913,12 +931,16 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit code""" |$buffer.append("["); |if ($array.numElements() > 0) { - | if (!$array.isNullAt(0)) { + | if ($array.isNullAt(0)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append($elementToStringFunc(${CodeGenerator.getValue(array, et, "0")})); | } | for (int $loopIndex = 1; $loopIndex < $array.numElements(); $loopIndex++) { | $buffer.append(","); - | if (!$array.isNullAt($loopIndex)) { + | if ($array.isNullAt($loopIndex)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append(" "); | $buffer.append($elementToStringFunc(${CodeGenerator.getValue(array, et, loopIndex)})); | } @@ -962,11 +984,13 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val getMapKeyArray = CodeGenerator.getValue(mapKeyArray, kt, loopIndex) val getMapValueArray = CodeGenerator.getValue(mapValueArray, vt, loopIndex) code""" - |$buffer.append("["); + |$buffer.append("$leftBracket"); |if ($map.numElements() > 0) { | $buffer.append($keyToStringFunc($getMapFirstKey)); | $buffer.append(" ->"); - | if (!$map.valueArray().isNullAt(0)) { + | if ($map.valueArray().isNullAt(0)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append(" "); | $buffer.append($valueToStringFunc($getMapFirstValue)); | } @@ -974,13 +998,15 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit | $buffer.append(", "); | $buffer.append($keyToStringFunc($getMapKeyArray)); | $buffer.append(" ->"); - | if (!$map.valueArray().isNullAt($loopIndex)) { + | if ($map.valueArray().isNullAt($loopIndex)) { + | ${outNullElem(buffer)} + | } else { | $buffer.append(" "); | $buffer.append($valueToStringFunc($getMapValueArray)); | } | } |} - |$buffer.append("]"); + |$buffer.append("$rightBracket"); """.stripMargin } @@ -996,7 +1022,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val javaType = JavaCode.javaType(ft) code""" |${if (i != 0) code"""$buffer.append(",");""" else EmptyBlock} - |if (!$row.isNullAt($i)) { + |if ($row.isNullAt($i)) { + | ${outNullElem(buffer)} + |} else { | ${if (i != 0) code"""$buffer.append(" ");""" else EmptyBlock} | | // Append $i field into the string buffer @@ -1015,9 +1043,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (classOf[UTF8StringBuilder].getName, buffer.code) :: Nil) code""" - |$buffer.append("["); + |$buffer.append("$leftBracket"); |$writeStructCode - |$buffer.append("]"); + |$buffer.append("$rightBracket"); """.stripMargin } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala index df806edbfda05..65fd43c924d08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckSuccess import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -49,12 +50,16 @@ import org.apache.spark.sql.types._ """, group = "agg_funcs", since = "2.0.0") -case class First(child: Expression, ignoreNullsExpr: Expression) +case class First(child: Expression, ignoreNulls: Boolean) extends DeclarativeAggregate with ExpectsInputTypes { - def this(child: Expression) = this(child, Literal.create(false, BooleanType)) + def this(child: Expression) = this(child, false) - override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil + def this(child: Expression, ignoreNullsExpr: Expression) = { + this(child, FirstLast.validateIgnoreNullExpr(ignoreNullsExpr, "first")) + } + + override def children: Seq[Expression] = child :: Nil override def nullable: Boolean = true @@ -71,16 +76,11 @@ case class First(child: Expression, ignoreNullsExpr: Expression) val defaultCheck = super.checkInputDataTypes() if (defaultCheck.isFailure) { defaultCheck - } else if (!ignoreNullsExpr.foldable) { - TypeCheckFailure( - s"The second argument of First must be a boolean literal, but got: ${ignoreNullsExpr.sql}") } else { TypeCheckSuccess } } - private def ignoreNulls: Boolean = ignoreNullsExpr.eval().asInstanceOf[Boolean] - private lazy val first = AttributeReference("first", child.dataType)() private lazy val valueSet = AttributeReference("valueSet", BooleanType)() @@ -120,3 +120,11 @@ case class First(child: Expression, ignoreNullsExpr: Expression) override def toString: String = s"$prettyName($child)${if (ignoreNulls) " ignore nulls"}" } + +object FirstLast { + def validateIgnoreNullExpr(exp: Expression, funcName: String): Boolean = exp match { + case Literal(b: Boolean, BooleanType) => b + case _ => throw new AnalysisException( + s"The second argument in $funcName should be a boolean literal.") + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index e55bced192f34..8d17a48a69f6f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckSuccess import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -49,12 +49,16 @@ import org.apache.spark.sql.types._ """, group = "agg_funcs", since = "2.0.0") -case class Last(child: Expression, ignoreNullsExpr: Expression) +case class Last(child: Expression, ignoreNulls: Boolean) extends DeclarativeAggregate with ExpectsInputTypes { - def this(child: Expression) = this(child, Literal.create(false, BooleanType)) + def this(child: Expression) = this(child, false) - override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil + def this(child: Expression, ignoreNullsExpr: Expression) = { + this(child, FirstLast.validateIgnoreNullExpr(ignoreNullsExpr, "last")) + } + + override def children: Seq[Expression] = child :: Nil override def nullable: Boolean = true @@ -71,16 +75,11 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) val defaultCheck = super.checkInputDataTypes() if (defaultCheck.isFailure) { defaultCheck - } else if (!ignoreNullsExpr.foldable) { - TypeCheckFailure( - s"The second argument of Last must be a boolean literal, but got: ${ignoreNullsExpr.sql}") } else { TypeCheckSuccess } } - private def ignoreNulls: Boolean = ignoreNullsExpr.eval().asInstanceOf[Boolean] - private lazy val last = AttributeReference("last", child.dataType)() private lazy val valueSet = AttributeReference("valueSet", BooleanType)() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index cf7cc3a5e16ff..563ce7133a3dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -563,8 +563,6 @@ case class WithFields( override def dataType: StructType = evalExpr.dataType.asInstanceOf[StructType] - override def foldable: Boolean = structExpr.foldable && valExprs.forall(_.foldable) - override def nullable: Boolean = structExpr.nullable override def prettyName: String = "with_fields" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 5c764956adf0a..7f6ad2467480d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -1406,6 +1406,7 @@ case class WidthBucket( override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType, DoubleType, DoubleType, LongType) override def dataType: DataType = LongType override def nullable: Boolean = true + override def prettyName: String = "width_bucket" override protected def nullSafeEval(input: Any, min: Any, max: Any, numBucket: Any): Any = { WidthBucket.computeBucketNumber( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 361bcd492965b..7cf2c73f9c941 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.objects import java.lang.reflect.{Method, Modifier} import scala.collection.JavaConverters._ -import scala.collection.mutable.Builder +import scala.collection.mutable.{Builder, IndexedSeq, WrappedArray} import scala.reflect.ClassTag import scala.util.Try @@ -755,6 +755,9 @@ case class MapObjects private( } private lazy val mapElements: Seq[_] => Any = customCollectionCls match { + case Some(cls) if classOf[WrappedArray[_]].isAssignableFrom(cls) => + // Scala WrappedArray + inputCollection => WrappedArray.make(executeFuncOnCollection(inputCollection).toArray) case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) => // Scala sequence executeFuncOnCollection(_).toSeq @@ -912,6 +915,20 @@ case class MapObjects private( val (initCollection, addElement, getResult): (String, String => String, String) = customCollectionCls match { + case Some(cls) if classOf[WrappedArray[_]].isAssignableFrom(cls) => + // Scala WrappedArray + val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()" + val builder = ctx.freshName("collectionBuilder") + ( + s""" + ${classOf[Builder[_, _]].getName} $builder = $getBuilder; + $builder.sizeHint($dataLength); + """, + (genValue: String) => s"$builder.$$plus$$eq($genValue);", + s"(${cls.getName}) ${classOf[WrappedArray[_]].getName}$$." + + s"MODULE$$.make(((${classOf[IndexedSeq[_]].getName})$builder" + + s".result()).toArray(scala.reflect.ClassTag$$.MODULE$$.Object()));" + ) case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) || classOf[scala.collection.Set[_]].isAssignableFrom(cls) => // Scala sequence or set diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 527618b8e2c5a..aa5cf4758564b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -202,125 +202,50 @@ trait PredicateHelper extends Logging { } /** - * Convert an expression into conjunctive normal form. - * Definition and algorithm: https://en.wikipedia.org/wiki/Conjunctive_normal_form - * CNF can explode exponentially in the size of the input expression when converting [[Or]] - * clauses. Use a configuration [[SQLConf.MAX_CNF_NODE_COUNT]] to prevent such cases. - * - * @param condition to be converted into CNF. - * @return the CNF result as sequence of disjunctive expressions. If the number of expressions - * exceeds threshold on converting `Or`, `Seq.empty` is returned. + * Returns a filter that its reference is a subset of `outputSet` and it contains the maximum + * constraints from `condition`. This is used for predicate pushdown. + * When there is no such filter, `None` is returned. */ - protected def conjunctiveNormalForm( + protected def extractPredicatesWithinOutputSet( condition: Expression, - groupExpsFunc: Seq[Expression] => Seq[Expression]): Seq[Expression] = { - val postOrderNodes = postOrderTraversal(condition) - val resultStack = new mutable.Stack[Seq[Expression]] - val maxCnfNodeCount = SQLConf.get.maxCnfNodeCount - // Bottom up approach to get CNF of sub-expressions - while (postOrderNodes.nonEmpty) { - val cnf = postOrderNodes.pop() match { - case _: And => - val right = resultStack.pop() - val left = resultStack.pop() - left ++ right - case _: Or => - // For each side, there is no need to expand predicates of the same references. - // So here we can aggregate predicates of the same qualifier as one single predicate, - // for reducing the size of pushed down predicates and corresponding codegen. - val right = groupExpsFunc(resultStack.pop()) - val left = groupExpsFunc(resultStack.pop()) - // Stop the loop whenever the result exceeds the `maxCnfNodeCount` - if (left.size * right.size > maxCnfNodeCount) { - logInfo(s"As the result size exceeds the threshold $maxCnfNodeCount. " + - "The CNF conversion is skipped and returning Seq.empty now. To avoid this, you can " + - s"raise the limit ${SQLConf.MAX_CNF_NODE_COUNT.key}.") - return Seq.empty - } else { - for { x <- left; y <- right } yield Or(x, y) - } - case other => other :: Nil + outputSet: AttributeSet): Option[Expression] = condition match { + case And(left, right) => + val leftResultOptional = extractPredicatesWithinOutputSet(left, outputSet) + val rightResultOptional = extractPredicatesWithinOutputSet(right, outputSet) + (leftResultOptional, rightResultOptional) match { + case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) + case (Some(leftResult), None) => Some(leftResult) + case (None, Some(rightResult)) => Some(rightResult) + case _ => None } - resultStack.push(cnf) - } - if (resultStack.length != 1) { - logWarning("The length of CNF conversion result stack is supposed to be 1. There might " + - "be something wrong with CNF conversion.") - return Seq.empty - } - resultStack.top - } - - /** - * Convert an expression to conjunctive normal form when pushing predicates through Join, - * when expand predicates, we can group by the qualifier avoiding generate unnecessary - * expression to control the length of final result since there are multiple tables. - * - * @param condition condition need to be converted - * @return the CNF result as sequence of disjunctive expressions. If the number of expressions - * exceeds threshold on converting `Or`, `Seq.empty` is returned. - */ - def CNFWithGroupExpressionsByQualifier(condition: Expression): Seq[Expression] = { - conjunctiveNormalForm(condition, (expressions: Seq[Expression]) => - expressions.groupBy(_.references.map(_.qualifier)).map(_._2.reduceLeft(And)).toSeq) - } - - /** - * Convert an expression to conjunctive normal form for predicate pushdown and partition pruning. - * When expanding predicates, this method groups expressions by their references for reducing - * the size of pushed down predicates and corresponding codegen. In partition pruning strategies, - * we split filters by [[splitConjunctivePredicates]] and partition filters by judging if it's - * references is subset of partCols, if we combine expressions group by reference when expand - * predicate of [[Or]], it won't impact final predicate pruning result since - * [[splitConjunctivePredicates]] won't split [[Or]] expression. - * - * @param condition condition need to be converted - * @return the CNF result as sequence of disjunctive expressions. If the number of expressions - * exceeds threshold on converting `Or`, `Seq.empty` is returned. - */ - def CNFWithGroupExpressionsByReference(condition: Expression): Seq[Expression] = { - conjunctiveNormalForm(condition, (expressions: Seq[Expression]) => - expressions.groupBy(e => AttributeSet(e.references)).map(_._2.reduceLeft(And)).toSeq) - } - /** - * Iterative post order traversal over a binary tree built by And/Or clauses with two stacks. - * For example, a condition `(a And b) Or c`, the postorder traversal is - * (`a`,`b`, `And`, `c`, `Or`). - * Following is the complete algorithm. After step 2, we get the postorder traversal in - * the second stack. - * 1. Push root to first stack. - * 2. Loop while first stack is not empty - * 2.1 Pop a node from first stack and push it to second stack - * 2.2 Push the children of the popped node to first stack - * - * @param condition to be traversed as binary tree - * @return sub-expressions in post order traversal as a stack. - * The first element of result stack is the leftmost node. - */ - private def postOrderTraversal(condition: Expression): mutable.Stack[Expression] = { - val stack = new mutable.Stack[Expression] - val result = new mutable.Stack[Expression] - stack.push(condition) - while (stack.nonEmpty) { - val node = stack.pop() - node match { - case Not(a And b) => stack.push(Or(Not(a), Not(b))) - case Not(a Or b) => stack.push(And(Not(a), Not(b))) - case Not(Not(a)) => stack.push(a) - case a And b => - result.push(node) - stack.push(a) - stack.push(b) - case a Or b => - result.push(node) - stack.push(a) - stack.push(b) - case _ => - result.push(node) + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // condition: (a1 AND a2) OR (b1 AND b2), + // outputSet: AttributeSet(a1, b1) + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + case Or(left, right) => + for { + lhs <- extractPredicatesWithinOutputSet(left, outputSet) + rhs <- extractPredicatesWithinOutputSet(right, outputSet) + } yield Or(lhs, rhs) + + // Here we assume all the `Not` operators is already below all the `And` and `Or` operators + // after the optimization rule `BooleanSimplification`, so that we don't need to handle the + // `Not` operators here. + case other => + if (other.references.subsetOf(outputSet)) { + Some(other) + } else { + None } - } - result } } 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 28924fac48eef..8eb7f463e049c 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 @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale -import java.util.regex.{MatchResult, Pattern} +import java.util.regex.{Matcher, MatchResult, Pattern} + +import scala.collection.mutable.ArrayBuffer import org.apache.commons.text.StringEscapeUtils @@ -410,22 +412,70 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio } } -object RegExpExtract { +object RegExpExtractBase { def checkGroupIndex(groupCount: Int, groupIndex: Int): Unit = { - if (groupCount < groupIndex) { + if (groupIndex < 0) { + throw new IllegalArgumentException("The specified group index cannot be less than zero") + } else if (groupCount < groupIndex) { throw new IllegalArgumentException( s"Regex group count is $groupCount, but the specified group index is $groupIndex") } } } +abstract class RegExpExtractBase + extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant { + def subject: Expression + def regexp: Expression + def idx: Expression + + // last regex in string, we will update the pattern iff regexp value changed. + @transient private var lastRegex: UTF8String = _ + // last regex pattern, we cache it for performance concern + @transient private var pattern: Pattern = _ + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) + override def children: Seq[Expression] = subject :: regexp :: idx :: Nil + + protected def getLastMatcher(s: Any, p: Any): Matcher = { + if (p != lastRegex) { + // regex value changed + lastRegex = p.asInstanceOf[UTF8String].clone() + pattern = Pattern.compile(lastRegex.toString) + } + pattern.matcher(s.toString) + } +} + /** * Extract a specific(idx) group identified by a Java regex. * * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. */ @ExpressionDescription( - usage = "_FUNC_(str, regexp[, idx]) - Extracts a group that matches `regexp`.", + usage = """ + _FUNC_(str, regexp[, idx]) - Extract the first string in the `str` that match the `regexp` + expression and corresponding to the regex group index. + """, + arguments = """ + Arguments: + * str - a string expression. + * regexp - a string representing a regular 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 + "^\\abc$". + + There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to + fallback to the Spark 1.6 behavior regarding string literal parsing. For example, + if the config is enabled, the `regexp` that can match "\abc" is "^\abc$". + * idx - an integer expression that representing the group index. The regex maybe contains + multiple groups. `idx` indicates which regex group to extract. The group index should + be non-negative. The minimum value of `idx` is 0, which means matching the entire + regular expression. If `idx` is not specified, the default group index value is 1. The + `idx` parameter is the Java regex Matcher group() method index. + """, examples = """ Examples: > SELECT _FUNC_('100-200', '(\\d+)-(\\d+)', 1); @@ -433,27 +483,17 @@ object RegExpExtract { """, since = "1.5.0") case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression) - extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant { + extends RegExpExtractBase { def this(s: Expression, r: Expression) = this(s, r, Literal(1)) - // last regex in string, we will update the pattern iff regexp value changed. - @transient private var lastRegex: UTF8String = _ - // last regex pattern, we cache it for performance concern - @transient private var pattern: Pattern = _ - override def nullSafeEval(s: Any, p: Any, r: Any): Any = { - if (!p.equals(lastRegex)) { - // regex value changed - lastRegex = p.asInstanceOf[UTF8String].clone() - pattern = Pattern.compile(lastRegex.toString) - } - val m = pattern.matcher(s.toString) + val m = getLastMatcher(s, p) if (m.find) { val mr: MatchResult = m.toMatchResult val index = r.asInstanceOf[Int] - RegExpExtract.checkGroupIndex(mr.groupCount, index) + RegExpExtractBase.checkGroupIndex(mr.groupCount, index) val group = mr.group(index) - if (group == null) { // Pattern matched, but not optional group + if (group == null) { // Pattern matched, but it's an optional group UTF8String.EMPTY_UTF8 } else { UTF8String.fromString(group) @@ -464,13 +504,11 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio } override def dataType: DataType = StringType - override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) - override def children: Seq[Expression] = subject :: regexp :: idx :: Nil override def prettyName: String = "regexp_extract" override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val classNamePattern = classOf[Pattern].getCanonicalName - val classNameRegExpExtract = classOf[RegExpExtract].getCanonicalName + val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName val matcher = ctx.freshName("matcher") val matchResult = ctx.freshName("matchResult") @@ -494,7 +532,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio $termPattern.matcher($subject.toString()); if ($matcher.find()) { java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); - $classNameRegExpExtract.checkGroupIndex($matchResult.groupCount(), $idx); + $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx); if ($matchResult.group($idx) == null) { ${ev.value} = UTF8String.EMPTY_UTF8; } else { @@ -508,3 +546,105 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio }) } } + +/** + * Extract all specific(idx) groups identified by a Java regex. + * + * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. + */ +@ExpressionDescription( + usage = """ + _FUNC_(str, regexp[, idx]) - Extract all strings in the `str` that match the `regexp` + expression and corresponding to the regex group index. + """, + arguments = """ + Arguments: + * str - a string expression. + * regexp - a string representing a regular 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 + "^\\abc$". + + There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to + fallback to the Spark 1.6 behavior regarding string literal parsing. For example, + if the config is enabled, the `regexp` that can match "\abc" is "^\abc$". + * idx - an integer expression that representing the group index. The regex may contains + multiple groups. `idx` indicates which regex group to extract. The group index should + be non-negative. The minimum value of `idx` is 0, which means matching the entire + regular expression. If `idx` is not specified, the default group index value is 1. The + `idx` parameter is the Java regex Matcher group() method index. + """, + examples = """ + Examples: + > SELECT _FUNC_('100-200, 300-400', '(\\d+)-(\\d+)', 1); + ["100","300"] + """, + since = "3.1.0") +case class RegExpExtractAll(subject: Expression, regexp: Expression, idx: Expression) + extends RegExpExtractBase { + def this(s: Expression, r: Expression) = this(s, r, Literal(1)) + + override def nullSafeEval(s: Any, p: Any, r: Any): Any = { + val m = getLastMatcher(s, p) + val matchResults = new ArrayBuffer[UTF8String]() + while(m.find) { + val mr: MatchResult = m.toMatchResult + val index = r.asInstanceOf[Int] + RegExpExtractBase.checkGroupIndex(mr.groupCount, index) + val group = mr.group(index) + if (group == null) { // Pattern matched, but it's an optional group + matchResults += UTF8String.EMPTY_UTF8 + } else { + matchResults += UTF8String.fromString(group) + } + } + + new GenericArrayData(matchResults.toArray.asInstanceOf[Array[Any]]) + } + + override def dataType: DataType = ArrayType(StringType) + override def prettyName: String = "regexp_extract_all" + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val classNamePattern = classOf[Pattern].getCanonicalName + val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName + val arrayClass = classOf[GenericArrayData].getName + val matcher = ctx.freshName("matcher") + val matchResult = ctx.freshName("matchResult") + val matchResults = ctx.freshName("matchResults") + + val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex") + val termPattern = ctx.addMutableState(classNamePattern, "pattern") + + val setEvNotNull = if (nullable) { + s"${ev.isNull} = false;" + } else { + "" + } + nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => { + s""" + | if (!$regexp.equals($termLastRegex)) { + | // regex value changed + | $termLastRegex = $regexp.clone(); + | $termPattern = $classNamePattern.compile($termLastRegex.toString()); + | } + | java.util.regex.Matcher $matcher = $termPattern.matcher($subject.toString()); + | java.util.ArrayList $matchResults = new java.util.ArrayList(); + | while ($matcher.find()) { + | java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); + | $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx); + | if ($matchResult.group($idx) == null) { + | $matchResults.add(UTF8String.EMPTY_UTF8); + | } else { + | $matchResults.add(UTF8String.fromString($matchResult.group($idx))); + | } + | } + | ${ev.value} = + | new $arrayClass($matchResults.toArray(new UTF8String[$matchResults.size()])); + | $setEvNotNull + """ + }) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 7f69180e87e7e..da76f8eeaf350 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -27,11 +27,12 @@ import com.fasterxml.jackson.core._ import org.apache.spark.SparkUpgradeException import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, StructFilters} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils @@ -42,7 +43,8 @@ import org.apache.spark.util.Utils class JacksonParser( schema: DataType, val options: JSONOptions, - allowArrayAsStructs: Boolean) extends Logging { + allowArrayAsStructs: Boolean, + filters: Seq[Filter] = Seq.empty) extends Logging { import JacksonUtils._ import com.fasterxml.jackson.core.JsonToken._ @@ -85,8 +87,13 @@ class JacksonParser( private def makeStructRootConverter(st: StructType): JsonParser => Iterable[InternalRow] = { val elementConverter = makeConverter(st) val fieldConverters = st.map(_.dataType).map(makeConverter).toArray + val jsonFilters = if (SQLConf.get.jsonFilterPushDown) { + new JsonFilters(filters, st) + } else { + new NoopFilters + } (parser: JsonParser) => parseJsonToken[Iterable[InternalRow]](parser, st) { - case START_OBJECT => Some(convertObject(parser, st, fieldConverters)) + case START_OBJECT => convertObject(parser, st, fieldConverters, jsonFilters) // SPARK-3308: support reading top level JSON arrays and take every element // in such an array as a row // @@ -146,7 +153,7 @@ class JacksonParser( // val st = at.elementType.asInstanceOf[StructType] val fieldConverters = st.map(_.dataType).map(makeConverter).toArray - Some(InternalRow(new GenericArrayData(Seq(convertObject(parser, st, fieldConverters))))) + Some(InternalRow(new GenericArrayData(convertObject(parser, st, fieldConverters).toArray))) } } @@ -291,7 +298,7 @@ class JacksonParser( case st: StructType => val fieldConverters = st.map(_.dataType).map(makeConverter).toArray (parser: JsonParser) => parseJsonToken[InternalRow](parser, dataType) { - case START_OBJECT => convertObject(parser, st, fieldConverters) + case START_OBJECT => convertObject(parser, st, fieldConverters).get } case at: ArrayType => @@ -375,15 +382,19 @@ class JacksonParser( private def convertObject( parser: JsonParser, schema: StructType, - fieldConverters: Array[ValueConverter]): InternalRow = { + fieldConverters: Array[ValueConverter], + structFilters: StructFilters = new NoopFilters()): Option[InternalRow] = { val row = new GenericInternalRow(schema.length) var badRecordException: Option[Throwable] = None + var skipRow = false - while (nextUntil(parser, JsonToken.END_OBJECT)) { + structFilters.reset() + while (!skipRow && nextUntil(parser, JsonToken.END_OBJECT)) { schema.getFieldIndex(parser.getCurrentName) match { case Some(index) => try { row.update(index, fieldConverters(index).apply(parser)) + skipRow = structFilters.skipRow(row, index) } catch { case e: SparkUpgradeException => throw e case NonFatal(e) => @@ -395,8 +406,10 @@ class JacksonParser( } } - if (badRecordException.isEmpty) { - row + if (skipRow) { + None + } else if (badRecordException.isEmpty) { + Some(row) } else { throw PartialResultException(row, badRecordException.get) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala new file mode 100644 index 0000000000000..d6adbe83584e3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonFilters.scala @@ -0,0 +1,157 @@ +/* + * 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.catalyst.json + +import org.apache.spark.sql.catalyst.{InternalRow, StructFilters} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType + +/** + * The class provides API for applying pushed down source filters to rows with + * a struct schema parsed from JSON records. The class should be used in this way: + * 1. Before processing of the next row, `JacksonParser` (parser for short) resets the internal + * state of `JsonFilters` by calling the `reset()` method. + * 2. The parser reads JSON fields one-by-one in streaming fashion. It converts an incoming + * field value to the desired type from the schema. After that, it sets the value to an instance + * of `InternalRow` at the position according to the schema. Order of parsed JSON fields can + * be different from the order in the schema. + * 3. Per every JSON field of the top-level JSON object, the parser calls `skipRow` by passing + * an `InternalRow` in which some of fields can be already set, and the position of the JSON + * field according to the schema. + * 3.1 `skipRow` finds a group of predicates that refers to this JSON field. + * 3.2 Per each predicate from the group, `skipRow` decrements its reference counter. + * 3.2.1 If predicate reference counter becomes 0, it means that all predicate attributes have + * been already set in the internal row, and the predicate can be applied to it. `skipRow` + * invokes the predicate for the row. + * 3.3 `skipRow` applies predicates until one of them returns `false`. In that case, the method + * returns `true` to the parser. + * 3.4 If all predicates with zero reference counter return `true`, the final result of + * the method is `false` which tells the parser to not skip the row. + * 4. If the parser gets `true` from `JsonFilters.skipRow`, it must not call the method anymore + * for this internal row, and should go the step 1. + * + * Besides of `StructFilters` assumptions, `JsonFilters` assumes that: + * - `skipRow()` can be called for any valid index of the struct fields, + * and in any order. + * - After `skipRow()` returns `true`, the internal state of `JsonFilters` can be inconsistent, + * so, `skipRow()` must not be called for the current row anymore without `reset()`. + * + * @param pushedFilters The pushed down source filters. The filters should refer to + * the fields of the provided schema. + * @param schema The required schema of records from datasource files. + */ +class JsonFilters(pushedFilters: Seq[sources.Filter], schema: StructType) + extends StructFilters(pushedFilters, schema) { + + /** + * Stateful JSON predicate that keeps track of its dependent references in the + * current row via `refCount`. + * + * @param predicate The predicate compiled from pushed down source filters. + * @param totalRefs The total amount of all filters references which the predicate + * compiled from. + */ + case class JsonPredicate(predicate: BasePredicate, totalRefs: Int) { + // The current number of predicate references in the row that have been not set yet. + // When `refCount` reaches zero, the predicate has all dependencies are set, and can + // be applied to the row. + var refCount: Int = totalRefs + + def reset(): Unit = { + refCount = totalRefs + } + } + + // Predicates compiled from the pushed down filters. The predicates are grouped by their + // attributes. The i-th group contains predicates that refer to the i-th field of the given + // schema. A predicates can be placed to many groups if it has many attributes. For example: + // schema: i INTEGER, s STRING + // filters: IsNotNull("i"), AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc")) + // predicates: + // 0: Array(IsNotNull("i"), AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + // 1: Array(AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + private val predicates: Array[Array[JsonPredicate]] = { + val groupedPredicates = Array.fill(schema.length)(Array.empty[JsonPredicate]) + val groupedByRefSet: Map[Set[String], JsonPredicate] = filters + // Group filters that have the same set of references. For example: + // IsNotNull("i") -> Set("i"), AlwaysTrue -> Set(), + // Or(EqualTo("i", 0), StringStartsWith("s", "abc")) -> Set("i", "s") + // By grouping filters we could avoid tracking their state of references in the + // current row separately. + .groupBy(_.references.toSet) + // Combine all filters from the same group by `And` because all filters should + // return `true` to do not skip a row. The result is compiled to a predicate. + .map { case (refSet, refsFilters) => + (refSet, JsonPredicate(toPredicate(refsFilters), refSet.size)) + } + // Apply predicates w/o references like `AlwaysTrue` and `AlwaysFalse` to all fields. + // We cannot set such predicates to a particular position because skipRow() can + // be invoked for any index due to unpredictable order of JSON fields in JSON records. + val withLiterals: Map[Set[String], JsonPredicate] = groupedByRefSet.map { + case (refSet, pred) if refSet.isEmpty => + (schema.fields.map(_.name).toSet, pred.copy(totalRefs = 1)) + case others => others + } + // Build a map where key is only one field and value is seq of predicates refer to the field + // "i" -> Seq(AlwaysTrue, IsNotNull("i"), Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + // "s" -> Seq(AlwaysTrue, Or(EqualTo("i", 0), StringStartsWith("s", "abc"))) + val groupedByFields: Map[String, Seq[(String, JsonPredicate)]] = withLiterals.toSeq + .flatMap { case (refSet, pred) => refSet.map((_, pred)) } + .groupBy(_._1) + // Build the final array by converting keys of `groupedByFields` to their + // indexes in the provided schema. + groupedByFields.foreach { case (fieldName, fieldPredicates) => + val fieldIndex = schema.fieldIndex(fieldName) + groupedPredicates(fieldIndex) = fieldPredicates.map(_._2).toArray + } + groupedPredicates + } + + /** + * Applies predicates (compiled filters) associated with the row field value + * at the position `index` only if other predicates dependencies are already + * set in the given row. + * + * Note: If the function returns `true`, `refCount` of some predicates can be not decremented. + * + * @param row The row with fully or partially set values. + * @param index The index of already set value. + * @return `true` if at least one of applicable predicates (all dependent row values are set) + * return `false`. It returns `false` if all predicates return `true`. + */ + def skipRow(row: InternalRow, index: Int): Boolean = { + assert(0 <= index && index < schema.fields.length, + s"The index $index is out of the valid range [0, ${schema.fields.length}). " + + s"It must point out to a field of the schema: ${schema.catalogString}.") + var skip = false + for (pred <- predicates(index) if !skip) { + pred.refCount -= 1 + assert(pred.refCount >= 0, + s"Predicate reference counter cannot be negative but got ${pred.refCount}.") + skip = pred.refCount == 0 && !pred.predicate.eval(row) + } + skip + } + + /** + * Reset states of all predicates by re-initializing reference counters. + */ + override def reset(): Unit = predicates.foreach(_.foreach(_.reset)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 0c8666b72cace..b053bf6d61e6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -31,6 +31,20 @@ import org.apache.spark.sql.types._ object NestedColumnAliasing { def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { + /** + * This pattern is needed to support [[Filter]] plan cases like + * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]). + * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that + * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule. + */ + case Project(projectList, Filter(condition, child)) + if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) => + val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions + getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map { + case (nestedFieldToAlias, attrToAliases) => + NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases) + } + case Project(projectList, child) if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) => val exprCandidatesToPrune = projectList ++ child.expressions @@ -113,6 +127,8 @@ object NestedColumnAliasing { case _: Sample => true case _: RepartitionByExpression => true case _: Join => true + case _: Window => true + case _: Sort => true case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 33da482c4eea4..b8da954d938c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -51,8 +51,7 @@ abstract class Optimizer(catalogManager: CatalogManager) override protected val excludedOnceBatches: Set[String] = Set( "PartitionPruning", - "Extract Python UDFs", - "Push CNF predicate through join") + "Extract Python UDFs") protected def fixedPoint = FixedPoint( @@ -123,8 +122,9 @@ abstract class Optimizer(catalogManager: CatalogManager) rulesWithoutInferFiltersFromConstraints: _*) :: // Set strategy to Once to avoid pushing filter every time because we do not change the // join condition. - Batch("Push CNF predicate through join", Once, - PushCNFPredicateThroughJoin) :: Nil + Batch("Push extra predicate through join", fixedPoint, + PushExtraPredicateThroughJoin, + PushDownPredicates) :: Nil } val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) :: @@ -497,8 +497,8 @@ object LimitPushDown extends Rule[LogicalPlan] { // Note: right now Union means UNION ALL, which does not de-duplicate rows, so it is safe to // pushdown Limit through it. Once we add UNION DISTINCT, however, we will not be able to // pushdown Limit. - case LocalLimit(exp, Union(children)) => - LocalLimit(exp, Union(children.map(maybePushLocalLimit(exp, _)))) + case LocalLimit(exp, u: Union) => + LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _)))) // Add extra limits below OUTER JOIN. For LEFT OUTER and RIGHT OUTER JOIN we push limits to // the left and right sides, respectively. It's not safe to push limits below FULL OUTER // JOIN in the general case without a more invasive rewrite. @@ -556,15 +556,15 @@ object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Push down deterministic projection through UNION ALL - case p @ Project(projectList, Union(children)) => - assert(children.nonEmpty) + case p @ Project(projectList, u: Union) => + assert(u.children.nonEmpty) if (projectList.forall(_.deterministic)) { - val newFirstChild = Project(projectList, children.head) - val newOtherChildren = children.tail.map { child => - val rewrites = buildRewrites(children.head, child) + val newFirstChild = Project(projectList, u.children.head) + val newOtherChildren = u.children.tail.map { child => + val rewrites = buildRewrites(u.children.head, child) Project(projectList.map(pushToRight(_, rewrites)), child) } - Union(newFirstChild +: newOtherChildren) + u.copy(children = newFirstChild +: newOtherChildren) } else { p } @@ -928,19 +928,28 @@ object CombineUnions extends Rule[LogicalPlan] { } private def flattenUnion(union: Union, flattenDistinct: Boolean): Union = { + val topByName = union.byName + val topAllowMissingCol = union.allowMissingCol + val stack = mutable.Stack[LogicalPlan](union) val flattened = mutable.ArrayBuffer.empty[LogicalPlan] + // Note that we should only flatten the unions with same byName and allowMissingCol. + // Although we do `UnionCoercion` at analysis phase, we manually run `CombineUnions` + // in some places like `Dataset.union`. Flattening unions with different resolution + // rules (by position and by name) could cause incorrect results. while (stack.nonEmpty) { stack.pop() match { - case Distinct(Union(children)) if flattenDistinct => + case Distinct(Union(children, byName, allowMissingCol)) + if flattenDistinct && byName == topByName && allowMissingCol == topAllowMissingCol => stack.pushAll(children.reverse) - case Union(children) => + case Union(children, byName, allowMissingCol) + if byName == topByName && allowMissingCol == topAllowMissingCol => stack.pushAll(children.reverse) case child => flattened += child } } - Union(flattened.toSeq) + union.copy(children = flattened) } } @@ -966,14 +975,19 @@ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { } /** - * Removes Sort operation. This can happen: + * Removes Sort operations if they don't affect the final output ordering. + * Note that changes in the final output ordering may affect the file size (SPARK-32318). + * This rule handles the following cases: * 1) if the sort order is empty or the sort order does not have any reference * 2) if the child is already sorted - * 3) if there is another Sort operator separated by 0...n Project/Filter operators - * 4) if the Sort operator is within Join separated by 0...n Project/Filter operators only, - * and the Join conditions is deterministic - * 5) if the Sort operator is within GroupBy separated by 0...n Project/Filter operators only, - * and the aggregate function is order irrelevant + * 3) if there is another Sort operator separated by 0...n Project, Filter, Repartition or + * RepartitionByExpression (with deterministic expressions) operators + * 4) if the Sort operator is within Join separated by 0...n Project, Filter, Repartition or + * RepartitionByExpression (with deterministic expressions) operators only and the Join condition + * is deterministic + * 5) if the Sort operator is within GroupBy separated by 0...n Project, Filter, Repartition or + * RepartitionByExpression (with deterministic expressions) operators only and the aggregate + * function is order irrelevant */ object EliminateSorts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -999,6 +1013,8 @@ object EliminateSorts extends Rule[LogicalPlan] { private def canEliminateSort(plan: LogicalPlan): Boolean = plan match { case p: Project => p.projectList.forall(_.deterministic) case f: Filter => f.condition.deterministic + case r: RepartitionByExpression => r.partitionExpressions.forall(_.deterministic) + case _: Repartition => true case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala index 0299646150ff3..d3cdd71eafdb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala @@ -50,8 +50,8 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper wit override def conf: SQLConf = SQLConf.get def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case p @ Union(children) if children.exists(isEmptyLocalRelation) => - val newChildren = children.filterNot(isEmptyLocalRelation) + case p: Union if p.children.exists(isEmptyLocalRelation) => + val newChildren = p.children.filterNot(isEmptyLocalRelation) if (newChildren.isEmpty) { empty(p) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushExtraPredicateThroughJoin.scala similarity index 59% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushExtraPredicateThroughJoin.scala index 47e9527ead7c3..0ba2ce3106061 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushExtraPredicateThroughJoin.scala @@ -17,18 +17,20 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.{And, Expression, PredicateHelper} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag /** - * Try converting join condition to conjunctive normal form expression so that more predicates may - * be able to be pushed down. + * Try pushing down disjunctive join condition into left and right child. * To avoid expanding the join condition, the join condition will be kept in the original form even * when predicate pushdown happens. */ -object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { +object PushExtraPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { + + private val processedJoinConditionTag = TreeNodeTag[Expression]("processedJoinCondition") private def canPushThrough(joinType: JoinType): Boolean = joinType match { case _: InnerLike | LeftSemi | RightOuter | LeftOuter | LeftAnti | ExistenceJoin(_) => true @@ -38,22 +40,28 @@ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelpe def apply(plan: LogicalPlan): LogicalPlan = plan transform { case j @ Join(left, right, joinType, Some(joinCondition), hint) if canPushThrough(joinType) => - val predicates = CNFWithGroupExpressionsByQualifier(joinCondition) - if (predicates.isEmpty) { + val alreadyProcessed = j.getTagValue(processedJoinConditionTag).exists { condition => + condition.semanticEquals(joinCondition) + } + + lazy val filtersOfBothSide = splitConjunctivePredicates(joinCondition).filter { f => + f.deterministic && f.references.nonEmpty && + !f.references.subsetOf(left.outputSet) && !f.references.subsetOf(right.outputSet) + } + lazy val leftExtraCondition = + filtersOfBothSide.flatMap(extractPredicatesWithinOutputSet(_, left.outputSet)) + lazy val rightExtraCondition = + filtersOfBothSide.flatMap(extractPredicatesWithinOutputSet(_, right.outputSet)) + + if (alreadyProcessed || (leftExtraCondition.isEmpty && rightExtraCondition.isEmpty)) { j } else { - val pushDownCandidates = predicates.filter(_.deterministic) - lazy val leftFilterConditions = - pushDownCandidates.filter(_.references.subsetOf(left.outputSet)) - lazy val rightFilterConditions = - pushDownCandidates.filter(_.references.subsetOf(right.outputSet)) - lazy val newLeft = - leftFilterConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) + leftExtraCondition.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) lazy val newRight = - rightFilterConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) + rightExtraCondition.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) - joinType match { + val newJoin = joinType match { case _: InnerLike | LeftSemi => Join(newLeft, newRight, joinType, Some(joinCondition), hint) case RightOuter => @@ -63,6 +71,8 @@ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelpe case other => throw new IllegalStateException(s"Unexpected join type: $other") } - } + newJoin.setTagValue(processedJoinConditionTag, joinCondition) + newJoin + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index e5571069a7c41..af3a8fe684bb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Complete} +import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types.IntegerType @@ -81,10 +81,10 @@ import org.apache.spark.sql.types.IntegerType * COUNT(DISTINCT cat1) as cat1_cnt, * COUNT(DISTINCT cat2) as cat2_cnt, * SUM(value) FILTER (WHERE id > 1) AS total - * FROM - * data - * GROUP BY - * key + * FROM + * data + * GROUP BY + * key * }}} * * This translates to the following (pseudo) logical plan: @@ -93,7 +93,7 @@ import org.apache.spark.sql.types.IntegerType * key = ['key] * functions = [COUNT(DISTINCT 'cat1), * COUNT(DISTINCT 'cat2), - * sum('value) with FILTER('id > 1)] + * sum('value) FILTER (WHERE 'id > 1)] * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) * LocalTableScan [...] * }}} @@ -108,7 +108,7 @@ import org.apache.spark.sql.types.IntegerType * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) * Aggregate( * key = ['key, 'cat1, 'cat2, 'gid] - * functions = [sum('value) with FILTER('id > 1)] + * functions = [sum('value) FILTER (WHERE 'id > 1)] * output = ['key, 'cat1, 'cat2, 'gid, 'total]) * Expand( * projections = [('key, null, null, 0, cast('value as bigint), 'id), @@ -118,6 +118,49 @@ import org.apache.spark.sql.types.IntegerType * LocalTableScan [...] * }}} * + * Third example: aggregate function with distinct and filter clauses (in sql): + * {{{ + * SELECT + * COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt, + * COUNT(DISTINCT cat2) FILTER (WHERE id > 2) as cat2_cnt, + * SUM(value) FILTER (WHERE id > 3) AS total + * FROM + * data + * GROUP BY + * key + * }}} + * + * This translates to the following (pseudo) logical plan: + * {{{ + * Aggregate( + * key = ['key] + * functions = [COUNT(DISTINCT 'cat1) FILTER (WHERE 'id > 1), + * COUNT(DISTINCT 'cat2) FILTER (WHERE 'id > 2), + * sum('value) FILTER (WHERE 'id > 3)] + * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) + * LocalTableScan [...] + * }}} + * + * This rule rewrites this logical plan to the following (pseudo) logical plan: + * {{{ + * Aggregate( + * key = ['key] + * functions = [count(if (('gid = 1) and 'max_cond1) 'cat1 else null), + * count(if (('gid = 2) and 'max_cond2) 'cat2 else null), + * first(if (('gid = 0)) 'total else null) ignore nulls] + * output = ['key, 'cat1_cnt, 'cat2_cnt, 'total]) + * Aggregate( + * key = ['key, 'cat1, 'cat2, 'gid] + * functions = [max('cond1), max('cond2), sum('value) FILTER (WHERE 'id > 3)] + * output = ['key, 'cat1, 'cat2, 'gid, 'max_cond1, 'max_cond2, 'total]) + * Expand( + * projections = [('key, null, null, 0, null, null, cast('value as bigint), 'id), + * ('key, 'cat1, null, 1, 'id > 1, null, null, null), + * ('key, null, 'cat2, 2, null, 'id > 2, null, null)] + * output = ['key, 'cat1, 'cat2, 'gid, 'cond1, 'cond2, 'value, 'id]) + * LocalTableScan [...] + * }}} + * * The rule does the following things here: * 1. Expand the data. There are three aggregation groups in this query: * i. the non-distinct group; @@ -126,15 +169,24 @@ import org.apache.spark.sql.types.IntegerType * An expand operator is inserted to expand the child data for each group. The expand will null * out all unused columns for the given group; this must be done in order to ensure correctness * later on. Groups can by identified by a group id (gid) column added by the expand operator. + * If distinct group exists filter clause, the expand will calculate the filter and output it's + * result (e.g. cond1) which will be used to calculate the global conditions (e.g. max_cond1) + * equivalent to filter clauses. * 2. De-duplicate the distinct paths and aggregate the non-aggregate path. The group by clause of * this aggregate consists of the original group by clause, all the requested distinct columns * and the group id. Both de-duplication of distinct column and the aggregation of the * non-distinct group take advantage of the fact that we group by the group id (gid) and that we - * have nulled out all non-relevant columns the given group. + * have nulled out all non-relevant columns the given group. If distinct group exists filter + * clause, we will use max to aggregate the results (e.g. cond1) of the filter output in the + * previous step. These aggregate will output the global conditions (e.g. max_cond1) equivalent + * to filter clauses. * 3. Aggregating the distinct groups and combining this with the results of the non-distinct - * aggregation. In this step we use the group id to filter the inputs for the aggregate - * functions. The result of the non-distinct group are 'aggregated' by using the first operator, - * it might be more elegant to use the native UDAF merge mechanism for this in the future. + * aggregation. In this step we use the group id and the global condition to filter the inputs + * for the aggregate functions. If the global condition (e.g. max_cond1) is true, it means at + * least one row of a distinct value satisfies the filter. This distinct value should be included + * in the aggregate function. The result of the non-distinct group are 'aggregated' by using + * the first operator, it might be more elegant to use the native UDAF merge mechanism for this + * in the future. * * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and @@ -144,28 +196,24 @@ import org.apache.spark.sql.types.IntegerType */ object RewriteDistinctAggregates extends Rule[LogicalPlan] { - private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = { - val distinctAggs = exprs.flatMap { _.collect { - case ae: AggregateExpression if ae.isDistinct => ae - }} - // We need at least two distinct aggregates for this rule because aggregation - // strategy can handle a single distinct group. + private def mayNeedtoRewrite(a: Aggregate): Boolean = { + val aggExpressions = collectAggregateExprs(a) + val distinctAggs = aggExpressions.filter(_.isDistinct) + // We need at least two distinct aggregates or the single distinct aggregate group exists filter + // clause for this rule because aggregation strategy can handle a single distinct aggregate + // group without filter clause. // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a). - distinctAggs.size > 1 + distinctAggs.size > 1 || distinctAggs.exists(_.filter.isDefined) } def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a) + case a: Aggregate if mayNeedtoRewrite(a) => rewrite(a) } def rewrite(a: Aggregate): Aggregate = { - // Collect all aggregate expressions. - val aggExpressions = a.aggregateExpressions.flatMap { e => - e.collect { - case ae: AggregateExpression => ae - } - } + val aggExpressions = collectAggregateExprs(a) + val distinctAggs = aggExpressions.filter(_.isDistinct) // Extract distinct aggregate expressions. val distinctAggGroups = aggExpressions.filter(_.isDistinct).groupBy { e => @@ -184,8 +232,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { } } - // Aggregation strategy can handle queries with a single distinct group. - if (distinctAggGroups.size > 1) { + // Aggregation strategy can handle queries with a single distinct group without filter clause. + if (distinctAggGroups.size > 1 || distinctAggs.exists(_.filter.isDefined)) { // Create the attributes for the grouping id and the group by clause. val gid = AttributeReference("gid", IntegerType, nullable = false)() val groupByMap = a.groupingExpressions.collect { @@ -195,7 +243,13 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val groupByAttrs = groupByMap.map(_._2) // Functions used to modify aggregate functions and their inputs. - def evalWithinGroup(id: Literal, e: Expression) = If(EqualTo(gid, id), e, nullify(e)) + def evalWithinGroup(id: Literal, e: Expression, condition: Option[Expression]) = + if (condition.isDefined) { + If(And(EqualTo(gid, id), condition.get), e, nullify(e)) + } else { + If(EqualTo(gid, id), e, nullify(e)) + } + def patchAggregateFunctionChildren( af: AggregateFunction)( attrs: Expression => Option[Expression]): AggregateFunction = { @@ -207,13 +261,28 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair) val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2) + // Setup all the filters in distinct aggregate. + val (distinctAggFilters, distinctAggFilterAttrs, maxConds) = distinctAggs.collect { + case AggregateExpression(_, _, _, filter, _) if filter.isDefined => + val (e, attr) = expressionAttributePair(filter.get) + val aggregateExp = Max(attr).toAggregateExpression() + (e, attr, Alias(aggregateExp, attr.name)()) + }.unzip3 // Setup expand & aggregate operators for distinct aggregate expressions. val distinctAggChildAttrLookup = distinctAggChildAttrMap.toMap + val distinctAggFilterAttrLookup = distinctAggFilters.zip(maxConds.map(_.toAttribute)).toMap val distinctAggOperatorMap = distinctAggGroups.toSeq.zipWithIndex.map { case ((group, expressions), i) => val id = Literal(i + 1) + // Expand projection for filter + val filters = expressions.filter(_.filter.isDefined).map(_.filter.get) + val filterProjection = distinctAggFilters.map { + case e if filters.contains(e) => e + case e => nullify(e) + } + // Expand projection val projection = distinctAggChildren.map { case e if group.contains(e) => e @@ -224,12 +293,17 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val operators = expressions.map { e => val af = e.aggregateFunction val naf = patchAggregateFunctionChildren(af) { x => - distinctAggChildAttrLookup.get(x).map(evalWithinGroup(id, _)) + val condition = if (e.filter.isDefined) { + e.filter.map(distinctAggFilterAttrLookup.get(_)).get + } else { + None + } + distinctAggChildAttrLookup.get(x).map(evalWithinGroup(id, _, condition)) } - (e, e.copy(aggregateFunction = naf, isDistinct = false)) + (e, e.copy(aggregateFunction = naf, isDistinct = false, filter = None)) } - (projection, operators) + (projection ++ filterProjection, operators) } // Setup expand for the 'regular' aggregate expressions. @@ -257,7 +331,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // Select the result of the first aggregate in the last aggregate. val result = AggregateExpression( - aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute), Literal(true)), + aggregate.First(evalWithinGroup(regularGroupId, operator.toAttribute, None), true), mode = Complete, isDistinct = false) @@ -280,6 +354,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { Seq(a.groupingExpressions ++ distinctAggChildren.map(nullify) ++ Seq(regularGroupId) ++ + distinctAggFilters.map(nullify) ++ regularAggChildren) } else { Seq.empty[Seq[Expression]] @@ -297,7 +372,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // Construct the expand operator. val expand = Expand( regularAggProjection ++ distinctAggProjections, - groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ regularAggChildAttrMap.map(_._2), + groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ distinctAggFilterAttrs ++ + regularAggChildAttrMap.map(_._2), a.child) // Construct the first aggregate operator. This de-duplicates all the children of @@ -305,7 +381,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { val firstAggregateGroupBy = groupByAttrs ++ distinctAggChildAttrs :+ gid val firstAggregate = Aggregate( firstAggregateGroupBy, - firstAggregateGroupBy ++ regularAggOperatorMap.map(_._2), + firstAggregateGroupBy ++ maxConds ++ regularAggOperatorMap.map(_._2), expand) // Construct the second aggregate @@ -331,6 +407,13 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { } } + private def collectAggregateExprs(a: Aggregate): Seq[AggregateExpression] = { + // Collect all aggregate expressions. + a.aggregateExpressions.flatMap { _.collect { + case ae: AggregateExpression => ae + }} + } + private def nullify(e: Expression) = Literal.create(null, e.dataType) private def expressionAttributePair(e: Expression) = 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 29621e11e534c..fe99a8ea3cc12 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 @@ -1535,7 +1535,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitFirst(ctx: FirstContext): Expression = withOrigin(ctx) { val ignoreNullsExpr = ctx.IGNORE != null - First(expression(ctx.expression), Literal(ignoreNullsExpr)).toAggregateExpression() + First(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() } /** @@ -1543,7 +1543,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitLast(ctx: LastContext): Expression = withOrigin(ctx) { val ignoreNullsExpr = ctx.IGNORE != null - Last(expression(ctx.expression), Literal(ignoreNullsExpr)).toAggregateExpression() + Last(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() } /** @@ -2090,6 +2090,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * - from-to unit, for instance: interval '1-2' year to month. */ override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) { + Literal(parseIntervalLiteral(ctx), CalendarIntervalType) + } + + /** + * Create a [[CalendarInterval]] object + */ + protected def parseIntervalLiteral(ctx: IntervalContext): CalendarInterval = withOrigin(ctx) { if (ctx.errorCapturingMultiUnitsInterval != null) { val innerCtx = ctx.errorCapturingMultiUnitsInterval if (innerCtx.unitToUnitInterval != null) { @@ -2097,7 +2104,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", innerCtx.unitToUnitInterval) } - Literal(visitMultiUnitsInterval(innerCtx.multiUnitsInterval), CalendarIntervalType) + visitMultiUnitsInterval(innerCtx.multiUnitsInterval) } else if (ctx.errorCapturingUnitToUnitInterval != null) { val innerCtx = ctx.errorCapturingUnitToUnitInterval if (innerCtx.error1 != null || innerCtx.error2 != null) { @@ -2106,7 +2113,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", errorCtx) } - Literal(visitUnitToUnitInterval(innerCtx.body), CalendarIntervalType) + visitUnitToUnitInterval(innerCtx.body) } else { throw new ParseException("at least one time unit should be given for interval literal", ctx) } @@ -3508,6 +3515,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + val properties = ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues) + .getOrElse(Map.empty) + if (ctx.TEMPORARY != null && !properties.isEmpty) { + operationNotAllowed("TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW", ctx) + } + val viewType = if (ctx.TEMPORARY == null) { PersistedView } else if (ctx.GLOBAL != null) { @@ -3519,8 +3532,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitMultipartIdentifier(ctx.multipartIdentifier), userSpecifiedColumns, visitCommentSpecList(ctx.commentSpec()), - ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues) - .getOrElse(Map.empty), + properties, Option(source(ctx.query)), plan(ctx.query), ctx.EXISTS != null, @@ -3588,7 +3600,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } else { Seq(describeFuncName.getText) } - DescribeFunctionStatement(functionName, EXTENDED != null) + DescribeFunction(UnresolvedFunc(functionName), EXTENDED != null) } /** @@ -3603,8 +3615,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case Some(x) => throw new ParseException(s"SHOW $x FUNCTIONS not supported", ctx) } val pattern = Option(ctx.pattern).map(string(_)) - val functionName = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) - ShowFunctionsStatement(userScope, systemScope, pattern, functionName) + val unresolvedFuncOpt = Option(ctx.multipartIdentifier) + .map(visitMultipartIdentifier) + .map(UnresolvedFunc(_)) + ShowFunctions(unresolvedFuncOpt, userScope, systemScope, pattern) } /** @@ -3617,8 +3631,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitDropFunction(ctx: DropFunctionContext): LogicalPlan = withOrigin(ctx) { val functionName = visitMultipartIdentifier(ctx.multipartIdentifier) - DropFunctionStatement( - functionName, + DropFunction( + UnresolvedFunc(functionName), ctx.EXISTS != null, ctx.TEMPORARY != null) } @@ -3653,6 +3667,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.REPLACE != null) } + override def visitRefreshFunction(ctx: RefreshFunctionContext): LogicalPlan = withOrigin(ctx) { + val functionIdentifier = visitMultipartIdentifier(ctx.multipartIdentifier) + RefreshFunction(UnresolvedFunc(functionIdentifier)) + } + override def visitCommentNamespace(ctx: CommentNamespaceContext): LogicalPlan = withOrigin(ctx) { val comment = ctx.comment.getType match { case SqlBaseParser.NULL => "" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index fab282f15f215..343b499f7231f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -96,9 +96,6 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) - lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - lexer.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled - lexer.SQL_standard_keyword_behavior = conf.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 415ce46788119..5a994f1ad0a39 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -21,8 +21,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.optimizer.JoinSelectionHelper import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf trait OperationHelper { type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) @@ -388,3 +390,37 @@ object PhysicalWindow { case _ => None } } + +object ExtractSingleColumnNullAwareAntiJoin extends JoinSelectionHelper with PredicateHelper { + + // TODO support multi column NULL-aware anti join in future. + // See. http://www.vldb.org/pvldb/vol2/vldb09-423.pdf Section 6 + // multi-column null aware anti join is much more complicated than single column ones. + + // streamedSideKeys, buildSideKeys + private type ReturnType = (Seq[Expression], Seq[Expression]) + + /** + * See. [SPARK-32290] + * LeftAnti(condition: Or(EqualTo(a=b), IsNull(EqualTo(a=b))) + * will almost certainly be planned as a Broadcast Nested Loop join, + * which is very time consuming because it's an O(M*N) calculation. + * But if it's a single column case O(M*N) calculation could be optimized into O(M) + * using hash lookup instead of loop lookup. + */ + def unapply(join: Join): Option[ReturnType] = join match { + case Join(left, right, LeftAnti, + Some(Or(e @ EqualTo(leftAttr: AttributeReference, rightAttr: AttributeReference), + IsNull(e2 @ EqualTo(_, _)))), _) + if SQLConf.get.optimizeNullAwareAntiJoin && + e.semanticEquals(e2) => + if (canEvaluate(leftAttr, left) && canEvaluate(rightAttr, right)) { + Some(Seq(leftAttr), Seq(rightAttr)) + } else if (canEvaluate(leftAttr, right) && canEvaluate(rightAttr, left)) { + Some(Seq(rightAttr), Seq(leftAttr)) + } else { + None + } + case _ => None + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index f1a363cca752e..c83f6a376aa8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -220,8 +220,18 @@ object Union { /** * Logical plan for unioning two plans, without a distinct. This is UNION ALL in SQL. + * + * @param byName Whether resolves columns in the children by column names. + * @param allowMissingCol Allows missing columns in children query plans. If it is true, + * this function allows different set of column names between two Datasets. + * This can be set to true only if `byName` is true. */ -case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { +case class Union( + children: Seq[LogicalPlan], + byName: Boolean = false, + allowMissingCol: Boolean = false) extends LogicalPlan { + assert(!allowMissingCol || byName, "`allowMissingCol` can be true only if `byName` is true.") + override def maxRows: Option[Long] = { if (children.exists(_.maxRows.isEmpty)) { None @@ -271,7 +281,7 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { child.output.zip(children.head.output).forall { case (l, r) => l.dataType.sameType(r.dataType) }) - children.length > 1 && childrenResolved && allChildrenCompatible + children.length > 1 && !(byName || allowMissingCol) && childrenResolved && allChildrenCompatible } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index b1129e741221b..19831a7b5ef84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -446,30 +446,6 @@ case class ShowColumnsStatement( */ case class ShowCurrentNamespaceStatement() extends ParsedStatement -/** - * A DESCRIBE FUNCTION statement, as parsed from SQL - */ -case class DescribeFunctionStatement( - functionName: Seq[String], - isExtended: Boolean) extends ParsedStatement - -/** - * SHOW FUNCTIONS statement, as parsed from SQL - */ -case class ShowFunctionsStatement( - userScope: Boolean, - systemScope: Boolean, - pattern: Option[String], - functionName: Option[Seq[String]]) extends ParsedStatement - -/** - * DROP FUNCTION statement, as parsed from SQL - */ -case class DropFunctionStatement( - functionName: Seq[String], - ifExists: Boolean, - isTemp: Boolean) extends ParsedStatement - /** * CREATE FUNCTION statement, as parsed from SQL */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index b4120d9f64cc5..70e03c23fd115 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -516,3 +516,38 @@ case class CommentOnNamespace(child: LogicalPlan, comment: String) extends Comma case class CommentOnTable(child: LogicalPlan, comment: String) extends Command { override def children: Seq[LogicalPlan] = child :: Nil } + +/** + * The logical plan of the REFRESH FUNCTION command that works for v2 catalogs. + */ +case class RefreshFunction(child: LogicalPlan) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil +} + +/** + * The logical plan of the DESCRIBE FUNCTION command that works for v2 catalogs. + */ +case class DescribeFunction(child: LogicalPlan, isExtended: Boolean) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil +} + +/** + * The logical plan of the DROP FUNCTION command that works for v2 catalogs. + */ +case class DropFunction( + child: LogicalPlan, + ifExists: Boolean, + isTemp: Boolean) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil +} + +/** + * The logical plan of the SHOW FUNCTIONS command that works for v2 catalogs. + */ +case class ShowFunctions( + child: Option[LogicalPlan], + userScope: Boolean, + systemScope: Boolean, + pattern: Option[String]) extends Command { + override def children: Seq[LogicalPlan] = child.toSeq +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 5de06af6af12a..f02b2d08c0935 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -296,7 +296,9 @@ private object DateTimeFormatterHelper { // unchecked `ArrayIndexOutOfBoundsException` by the `NumberPrinterParser` for formatting. It // makes the call side difficult to handle exceptions and easily leads to silent data change // because of the exceptions being suppressed. - Seq("y").map(_ * 11) + // SPARK-32424: The max year that we can actually handle is 6 digits, otherwise, it will + // overflow + Seq("y").map(_ * 7) }.toSet /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index 24fe9fa42c19f..1a78422e57a4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -186,7 +186,8 @@ object RebaseDateTime { .setTimeZone(TimeZoneUTC) .setDate(localDate.getYear, localDate.getMonthValue - 1, localDate.getDayOfMonth) .build() - Math.toIntExact(Math.floorDiv(utcCal.getTimeInMillis, MILLIS_PER_DAY)) + assert(utcCal.getTimeInMillis % MILLIS_PER_DAY == 0) + Math.toIntExact(utcCal.getTimeInMillis / MILLIS_PER_DAY) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 8db95044359c3..6832d1a9954fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -24,7 +24,6 @@ import java.time.format.{DateTimeFormatter, DateTimeParseException} import java.time.temporal.ChronoField.MICRO_OF_SECOND import java.time.temporal.TemporalQueries import java.util.{Calendar, GregorianCalendar, Locale, TimeZone} -import java.util.concurrent.TimeUnit.SECONDS import org.apache.commons.lang3.time.FastDateFormat @@ -83,7 +82,7 @@ class Iso8601TimestampFormatter( val epochSeconds = zonedDateTime.toEpochSecond val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) - Math.addExact(SECONDS.toMicros(epochSeconds), microsOfSecond) + Math.addExact(Math.multiplyExact(epochSeconds, MICROS_PER_SECOND), microsOfSecond) } catch checkParsedDiff(s, legacyFormatter.parse) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index d90804f4b6ff6..2ee760d4f60b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.connector.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} @@ -107,6 +107,14 @@ private[sql] object CatalogV2Implicits { throw new AnalysisException( s"$quoted is not a valid TableIdentifier as it has more than 2 name parts.") } + + def asFunctionIdentifier: FunctionIdentifier = ident.namespace() match { + case ns if ns.isEmpty => FunctionIdentifier(ident.name()) + case Array(dbName) => FunctionIdentifier(ident.name(), Some(dbName)) + case _ => + throw new AnalysisException( + s"$quoted is not a valid FunctionIdentifier as it has more than 2 name parts.") + } } implicit class MultipartIdentifierHelper(parts: Seq[String]) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 10c15747ec4ce..d8cdecce0d172 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} /** @@ -155,4 +155,28 @@ private[sql] trait LookupCatalog extends Logging { None } } + + def parseSessionCatalogFunctionIdentifier(nameParts: Seq[String]): FunctionIdentifier = { + if (nameParts.length == 1 && catalogManager.v1SessionCatalog.isTempFunction(nameParts.head)) { + return FunctionIdentifier(nameParts.head) + } + + nameParts match { + case SessionCatalogAndIdentifier(_, ident) => + if (nameParts.length == 1) { + // If there is only one name part, it means the current catalog is the session catalog. + // Here we don't fill the default database, to keep the error message unchanged for + // v1 commands. + FunctionIdentifier(nameParts.head, None) + } else { + ident.namespace match { + case Array(db) => FunctionIdentifier(ident.name, Some(db)) + case _ => + throw new AnalysisException(s"Unsupported function name '$ident'") + } + } + + case _ => throw new AnalysisException("function is only supported in v1 catalog") + } + } } 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 9be0497e46603..a69f09bdced17 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 @@ -545,19 +545,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val MAX_CNF_NODE_COUNT = - buildConf("spark.sql.optimizer.maxCNFNodeCount") - .internal() - .doc("Specifies the maximum allowable number of conjuncts in the result of CNF " + - "conversion. If the conversion exceeds the threshold, an empty sequence is returned. " + - "For example, CNF conversion of (a && b) || (c && d) generates " + - "four conjuncts (a || c) && (a || d) && (b || c) && (b || d).") - .version("3.1.0") - .intConf - .checkValue(_ >= 0, - "The depth of the maximum rewriting conjunction normal form must be positive.") - .createWithDefault(128) - val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals") .internal() .doc("When true, string literals (including regex patterns) remain escaped in our SQL " + @@ -1249,9 +1236,9 @@ object SQLConf { val STATE_STORE_FORMAT_VALIDATION_ENABLED = buildConf("spark.sql.streaming.stateStore.formatValidation.enabled") .internal() - .doc("When true, check if the UnsafeRow from the state store is valid or not when running " + - "streaming queries. This can happen if the state store format has been changed. Note, " + - "the feature is only effective in the build-in HDFS state store provider now.") + .doc("When true, check if the data from state store is valid or not when running streaming " + + "queries. This can happen if the state store format has been changed. Note, the feature " + + "is only effective in the build-in HDFS state store provider now.") .version("3.1.0") .booleanConf .createWithDefault(true) @@ -1723,9 +1710,9 @@ object SQLConf { val SESSION_LOCAL_TIMEZONE = buildConf("spark.sql.session.timeZone") .doc("The ID of session local timezone in the format of either region-based zone IDs or " + "zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. " + - "Zone offsets must be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. " + - "Also 'UTC' and 'Z' are supported as aliases of '+00:00'. Other short names are not " + - "recommended to use because they can be ambiguous.") + "Zone offsets must be in the format '(+|-)HH', '(+|-)HH:mm' or '(+|-)HH:mm:ss', e.g '-08', " + + "'+01:00' or '-13:33:33'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. Other " + + "short names are not recommended to use because they can be ambiguous.") .version("2.2.0") .stringConf .checkValue(isValidTimezone, s"Cannot resolve the given timezone with" + @@ -2543,6 +2530,18 @@ object SQLConf { .booleanConf .createWithDefault(true) + val JSON_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.json.filterPushdown.enabled") + .doc("When true, enable filter pushdown to JSON datasource.") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + + val AVRO_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.avro.filterPushdown.enabled") + .doc("When true, enable filter pushdown to Avro datasource.") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + val ADD_PARTITION_BATCH_SIZE = buildConf("spark.sql.addPartitionInBatch.size") .internal() @@ -2645,26 +2644,63 @@ object SQLConf { .booleanConf .createWithDefault(true) - val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED = - buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled") + val COALESCE_BUCKETS_IN_JOIN_ENABLED = + buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled") .doc("When true, if two bucketed tables with the different number of buckets are joined, " + "the side with a bigger number of buckets will be coalesced to have the same number " + - "of buckets as the other side. Bucket coalescing is applied only to sort-merge joins " + - "and only when the bigger number of buckets is divisible by the smaller number of buckets.") + "of buckets as the other side. Bigger number of buckets is divisible by the smaller " + + "number of buckets. Bucket coalescing is applied to sort-merge joins and " + + "shuffled hash join. Note: Coalescing bucketed table can avoid unnecessary shuffling " + + "in join, but it also reduces parallelism and could possibly cause OOM for " + + "shuffled hash join.") .version("3.1.0") .booleanConf .createWithDefault(false) - val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO = - buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio") + val COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO = + buildConf("spark.sql.bucketing.coalesceBucketsInJoin.maxBucketRatio") .doc("The ratio of the number of two buckets being coalesced should be less than or " + "equal to this value for bucket coalescing to be applied. This configuration only " + - s"has an effect when '${COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key}' is set to true.") + s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set to true.") .version("3.1.0") .intConf .checkValue(_ > 0, "The difference must be positive.") .createWithDefault(4) + val BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT = + buildConf("spark.sql.execution.broadcastHashJoin.outputPartitioningExpandLimit") + .internal() + .doc("The maximum number of partitionings that a HashPartitioning can be expanded to. " + + "This configuration is applicable only for BroadcastHashJoin inner joins and can be " + + "set to '0' to disable this feature.") + .version("3.1.0") + .intConf + .checkValue(_ >= 0, "The value must be non-negative.") + .createWithDefault(8) + + val OPTIMIZE_NULL_AWARE_ANTI_JOIN = + buildConf("spark.sql.optimizeNullAwareAntiJoin") + .internal() + .doc("When true, NULL-aware anti join execution will be planed into " + + "BroadcastHashJoinExec with flag isNullAwareAntiJoin enabled, " + + "optimized from O(M*N) calculation into O(M) calculation " + + "using Hash lookup instead of Looping lookup." + + "Only support for singleColumn NAAJ for now.") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + + val LEGACY_COMPLEX_TYPES_TO_STRING = + buildConf("spark.sql.legacy.castComplexTypesToString.enabled") + .internal() + .doc("When true, maps and structs are wrapped by [] in casting to strings, and " + + "NULL elements of structs/maps/arrays will be omitted while converting to strings. " + + "Otherwise, if this is false, which is the default, maps and structs are wrapped by {}, " + + "and NULL elements will be converted to \"null\".") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * @@ -2948,8 +2984,6 @@ class SQLConf extends Serializable with Logging { def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED) - def maxCnfNodeCount: Int = getConf(MAX_CNF_NODE_COUNT) - def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS) def fileCompressionFactor: Double = getConf(FILE_COMPRESSION_FACTOR) @@ -2975,6 +3009,9 @@ class SQLConf extends Serializable with Logging { LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY)) } + def broadcastHashJoinOutputPartitioningExpandLimit: Int = + getConf(BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT) + /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. @@ -3255,11 +3292,25 @@ class SQLConf extends Serializable with Logging { def csvFilterPushDown: Boolean = getConf(CSV_FILTER_PUSHDOWN_ENABLED) + def jsonFilterPushDown: Boolean = getConf(JSON_FILTER_PUSHDOWN_ENABLED) + + def avroFilterPushDown: Boolean = getConf(AVRO_FILTER_PUSHDOWN_ENABLED) + def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) def legacyAllowCastNumericToTimestamp: Boolean = getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) + def metadataCacheTTL: Long = getConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS) + + def coalesceBucketsInJoinEnabled: Boolean = getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED) + + def coalesceBucketsInJoinMaxBucketRatio: Int = + getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO) + + def optimizeNullAwareAntiJoin: Boolean = + getConf(SQLConf.OPTIMIZE_NULL_AWARE_ANTI_JOIN) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 9618ff6062635..ca1074fcf6fc0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.internal import java.util.Locale +import java.util.concurrent.TimeUnit import org.apache.spark.util.Utils @@ -126,6 +127,16 @@ object StaticSQLConf { .toSequence .createOptional + val SPARK_CACHE_SERIALIZER = buildStaticConf("spark.sql.cache.serializer") + .doc("The name of a class that implements " + + "org.apache.spark.sql.columnar.CachedBatchSerializer. It will be used to " + + "translate SQL data into a format that can more efficiently be cached. The underlying " + + "API is subject to change so use with caution. Multiple classes cannot be specified. " + + "The class must have a no-arg constructor.") + .version("3.1.0") + .stringConf + .createWithDefault("org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer") + val QUERY_EXECUTION_LISTENERS = buildStaticConf("spark.sql.queryExecutionListeners") .doc("List of class names implementing QueryExecutionListener that will be automatically " + "added to newly created sessions. The classes should have either a no-arg constructor, " + @@ -226,4 +237,16 @@ object StaticSQLConf { .version("3.0.0") .intConf .createWithDefault(100) + + val METADATA_CACHE_TTL_SECONDS = buildStaticConf("spark.sql.metadataCacheTTLSeconds") + .doc("Time-to-live (TTL) value for the metadata caches: partition file metadata cache and " + + "session catalog cache. This configuration only has an effect when this value having " + + "a positive value (> 0). It also requires setting " + + s"'${StaticSQLConf.CATALOG_IMPLEMENTATION.key}' to `hive`, setting " + + s"'${SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key}' > 0 and setting " + + s"'${SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key}' to `true` " + + "to be applied to the partition file metadata cache.") + .version("3.1.0") + .timeConf(TimeUnit.SECONDS) + .createWithDefault(-1) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 27b5eec27281d..c83cd52250702 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -41,8 +41,38 @@ private[spark] object SchemaUtils { * @param caseSensitiveAnalysis whether duplication checks should be case sensitive or not */ def checkSchemaColumnNameDuplication( - schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { - checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis) + schema: DataType, + colType: String, + caseSensitiveAnalysis: Boolean = false): Unit = { + schema match { + case ArrayType(elementType, _) => + checkSchemaColumnNameDuplication(elementType, colType, caseSensitiveAnalysis) + case MapType(keyType, valueType, _) => + checkSchemaColumnNameDuplication(keyType, colType, caseSensitiveAnalysis) + checkSchemaColumnNameDuplication(valueType, colType, caseSensitiveAnalysis) + case structType: StructType => + val fields = structType.fields + checkColumnNameDuplication(fields.map(_.name), colType, caseSensitiveAnalysis) + fields.foreach { field => + checkSchemaColumnNameDuplication(field.dataType, colType, caseSensitiveAnalysis) + } + case _ => + } + } + + /** + * Checks if an input schema has duplicate column names. This throws an exception if the + * duplication exists. + * + * @param schema schema to check + * @param colType column type name, used in an exception message + * @param resolver resolver used to determine if two identifiers are equal + */ + def checkSchemaColumnNameDuplication( + schema: StructType, + colType: String, + resolver: Resolver): Unit = { + checkSchemaColumnNameDuplication(schema, colType, isCaseSensitiveAnalysis(resolver)) } // Returns true if a given resolver is case-sensitive diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala index 25699de33d717..385f749736846 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql -import org.scalatest.{FunSpec, Matchers} +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema} import org.apache.spark.sql.types._ -class RowTest extends FunSpec with Matchers { +class RowTest extends AnyFunSpec with Matchers { val schema = StructType( StructField("col1", StringType) :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/OrderedFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/OrderedFiltersSuite.scala new file mode 100644 index 0000000000000..b156cb52e921c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/OrderedFiltersSuite.scala @@ -0,0 +1,27 @@ +/* + * 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.catalyst + +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType + +class OrderedFiltersSuite extends StructFiltersSuite { + override def createFilters(filters: Seq[sources.Filter], schema: StructType): StructFilters = { + new OrderedFilters(filters, schema) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala index 3d41d02b23df5..082b01173e7b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala @@ -38,7 +38,7 @@ trait SQLKeywordUtils extends SQLHelper { } // each element is an array of 4 string: the keyword name, reserve or not in Spark ANSI mode, - // Spark non-ANSI mode, and the SQL standard. + // Spark default mode, and the SQL standard. val keywordsInDoc: Array[Array[String]] = { val docPath = { java.nio.file.Paths.get(sparkHome, "docs", "sql-ref-ansi-compliance.md").toFile @@ -135,6 +135,19 @@ trait SQLKeywordUtils extends SQLHelper { } val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode + + val nonReservedKeywordsInDefaultMode: Set[String] = { + val kwDef = """\s*[\|:]\s*([A-Z_]+)\s*""".r + parseAntlrGrammars("//--DEFAULT-NON-RESERVED-START", "//--DEFAULT-NON-RESERVED-END") { + // Parses a pattern, e.g., ` | AFTER` + case kwDef(symbol) => + if (symbolsToExpandIntoDifferentLiterals.contains(symbol)) { + symbolsToExpandIntoDifferentLiterals(symbol) + } else { + symbol :: Nil + } + } + } } class SQLKeywordSuite extends SparkFunSuite with SQLKeywordUtils { @@ -146,11 +159,32 @@ class SQLKeywordSuite extends SparkFunSuite with SQLKeywordUtils { } } - test("Spark keywords are documented correctly") { - val reservedKeywordsInDoc = keywordsInDoc.filter(_.apply(1) == "reserved").map(_.head).toSet - if (reservedKeywordsInAnsiMode != reservedKeywordsInDoc) { - val misImplemented = (reservedKeywordsInDoc -- reservedKeywordsInAnsiMode).toSeq.sorted - fail("Some keywords are documented as reserved but actually not: " + + test("Spark keywords are documented correctly under ANSI mode") { + // keywords under ANSI mode should either be reserved or non-reserved. + keywordsInDoc.map(_.apply(1)).foreach { desc => + assert(desc == "reserved" || desc == "non-reserved") + } + + val nonReservedInDoc = keywordsInDoc.filter(_.apply(1) == "non-reserved").map(_.head).toSet + if (nonReservedKeywordsInAnsiMode != nonReservedInDoc) { + val misImplemented = ((nonReservedInDoc -- nonReservedKeywordsInAnsiMode) ++ + (nonReservedKeywordsInAnsiMode -- nonReservedInDoc)).toSeq.sorted + fail("Some keywords are documented and implemented inconsistently: " + + misImplemented.mkString(", ")) + } + } + + test("Spark keywords are documented correctly under default mode") { + // keywords under default mode should either be strict-non-reserved or non-reserved. + keywordsInDoc.map(_.apply(2)).foreach { desc => + assert(desc == "strict-non-reserved" || desc == "non-reserved") + } + + val nonReservedInDoc = keywordsInDoc.filter(_.apply(2) == "non-reserved").map(_.head).toSet + if (nonReservedKeywordsInDefaultMode != nonReservedInDoc) { + val misImplemented = ((nonReservedInDoc -- nonReservedKeywordsInDefaultMode) ++ + (nonReservedKeywordsInDefaultMode -- nonReservedInDoc)).toSeq.sorted + fail("Some keywords are documented and implemented inconsistently: " + misImplemented.mkString(", ")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/StructFiltersSuite.scala similarity index 82% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/StructFiltersSuite.scala index 499bbaf452aee..3893b2b07c519 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/StructFiltersSuite.scala @@ -15,21 +15,23 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.csv +package org.apache.spark.sql.catalyst import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.sources import org.apache.spark.sql.sources.{AlwaysFalse, AlwaysTrue, Filter} import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.unsafe.types.UTF8String -class CSVFiltersSuite extends SparkFunSuite { +abstract class StructFiltersSuite extends SparkFunSuite { + + protected def createFilters(filters: Seq[sources.Filter], schema: StructType): StructFilters + test("filter to expression conversion") { val ref = BoundReference(0, IntegerType, true) def check(f: Filter, expr: Expression): Unit = { - assert(CSVFilters.filterToExpression(f, _ => Some(ref)).get === expr) + assert(StructFilters.filterToExpression(f, _ => Some(ref)).get === expr) } check(sources.AlwaysTrue, Literal(true)) @@ -58,13 +60,14 @@ class CSVFiltersSuite extends SparkFunSuite { test("skipping rows") { def check( - requiredSchema: String = "i INTEGER, d DOUBLE", - filters: Seq[Filter], - row: InternalRow, - pos: Int, - skip: Boolean): Unit = { - val csvFilters = new CSVFilters(filters, getSchema(requiredSchema)) - assert(csvFilters.skipRow(row, pos) === skip) + requiredSchema: String = "i INTEGER, d DOUBLE", + filters: Seq[Filter], + row: InternalRow, + pos: Int, + skip: Boolean): Unit = { + val structFilters = createFilters(filters, getSchema(requiredSchema)) + structFilters.reset() + assert(structFilters.skipRow(row, pos) === skip) } check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) @@ -121,13 +124,12 @@ class CSVFiltersSuite extends SparkFunSuite { sources.LessThan("i", 0) ) Seq(filters1 -> false, filters2 -> true).foreach { case (filters, skip) => - for (p <- 0 until 3) { - check( - requiredSchema = "i INTEGER, d DOUBLE, s STRING", - filters = filters, - row = InternalRow(10, 3.14, UTF8String.fromString("abc")), - pos = p, - skip = skip) + val schema = "i INTEGER, d DOUBLE, s STRING" + val row = InternalRow(10, 3.14, UTF8String.fromString("abc")) + val structFilters = createFilters(filters, getSchema(schema)) + structFilters.reset() + for { p <- 0 until 3 if !skip } { + assert(structFilters.skipRow(row, p) === skip, s"p = $p filters = $filters skip = $skip") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 166ffec44a60d..a99f7e2be6e7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -207,11 +207,6 @@ class AnalysisErrorSuite extends AnalysisTest { "FILTER (WHERE c > 1)"), "FILTER predicate specified, but aggregate is not an aggregate function" :: Nil) - errorTest( - "DISTINCT aggregate function with filter predicate", - CatalystSqlParser.parsePlan("SELECT count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"), - "DISTINCT and FILTER cannot be used in aggregate functions at the same time" :: Nil) - errorTest( "non-deterministic filter predicate in aggregate functions", CatalystSqlParser.parsePlan("SELECT count(a) FILTER (WHERE rand(int(c)) > 1) FROM TaBlE2"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala index a85ac3fc4d0b7..3dd38091051d8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala @@ -21,7 +21,7 @@ import java.io.File import java.net.URI import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils class AnalysisExternalCatalogSuite extends AnalysisTest with Matchers { private def getAnalyzer(externCatalog: ExternalCatalog, databasePath: File): Analyzer = { 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 c0be49af2107d..9c9f59d155695 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 @@ -23,10 +23,10 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import org.apache.log4j.Level -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.api.python.PythonEvalType -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -895,4 +895,27 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisError(testRelation2.select(RowNumber() + 1), Seq("Window function row_number() requires an OVER clause.")) } + + test("SPARK-32237: Hint in CTE") { + val plan = With( + Project( + Seq(UnresolvedAttribute("cte.a")), + UnresolvedRelation(TableIdentifier("cte")) + ), + Seq( + ( + "cte", + SubqueryAlias( + AliasIdentifier("cte"), + UnresolvedHint( + "REPARTITION", + Seq(Literal(3)), + Project(testRelation.output, testRelation) + ) + ) + ) + ) + ) + assertAnalysisSuccess(plan) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index c3e18c7f9557f..d5991ff10ce6c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -73,7 +73,7 @@ class DecimalPrecisionSuite extends AnalysisTest with BeforeAndAfter { Union(Project(Seq(Alias(left, "l")()), relation), Project(Seq(Alias(right, "r")()), relation)) val (l, r) = analyzer.execute(plan).collect { - case Union(Seq(child1, child2)) => (child1.output.head, child2.output.head) + case Union(Seq(child1, child2), _, _) => (child1.output.head, child2.output.head) }.head assert(l.dataType === expectedType) assert(r.dataType === expectedType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnionSuite.scala new file mode 100644 index 0000000000000..5c7ad0067a456 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnionSuite.scala @@ -0,0 +1,75 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types._ + +class ResolveUnionSuite extends AnalysisTest { + test("Resolve Union") { + val table1 = LocalRelation( + AttributeReference("i", IntegerType)(), + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("b", ByteType)(), + AttributeReference("d", DoubleType)()) + val table2 = LocalRelation( + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("b", ByteType)(), + AttributeReference("d", DoubleType)(), + AttributeReference("i", IntegerType)()) + val table3 = LocalRelation( + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("d", DoubleType)(), + AttributeReference("i", IntegerType)()) + val table4 = LocalRelation( + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("i", IntegerType)()) + + val rules = Seq(ResolveUnion) + val analyzer = new RuleExecutor[LogicalPlan] { + override val batches = Seq(Batch("Resolution", Once, rules: _*)) + } + + // By name resolution + val union1 = Union(table1 :: table2 :: Nil, true, false) + val analyzed1 = analyzer.execute(union1) + val projected1 = + Project(Seq(table2.output(3), table2.output(0), table2.output(1), table2.output(2)), table2) + val expected1 = Union(table1 :: projected1 :: Nil) + comparePlans(analyzed1, expected1) + + // Allow missing column + val union2 = Union(table1 :: table3 :: Nil, true, true) + val analyzed2 = analyzer.execute(union2) + val nullAttr1 = Alias(Literal(null, ByteType), "b")() + val projected2 = + Project(Seq(table2.output(3), table2.output(0), nullAttr1, table2.output(2)), table3) + val expected2 = Union(table1 :: projected2 :: Nil) + comparePlans(analyzed2, expected2) + + // Allow missing column + Allow missing column + val union3 = Union(union2 :: table4 :: Nil, true, true) + val analyzed3 = analyzer.execute(union3) + val nullAttr2 = Alias(Literal(null, DoubleType), "d")() + val projected3 = + Project(Seq(table2.output(3), table2.output(0), nullAttr1, nullAttr2), table4) + val expected3 = Union(table1 :: projected2 :: projected3 :: Nil) + comparePlans(analyzed3, expected3) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala index eed962cd0f69d..06ea531833a43 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala @@ -22,7 +22,7 @@ import java.io.File import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 4d88a8d7ee546..ad40cc010361c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -17,15 +17,19 @@ package org.apache.spark.sql.catalyst.catalog +import scala.concurrent.duration._ + +import org.scalatest.concurrent.Eventually + import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{Command, Range, SubqueryAlias, View} import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types._ class InMemorySessionCatalogSuite extends SessionCatalogSuite { @@ -45,7 +49,7 @@ class InMemorySessionCatalogSuite extends SessionCatalogSuite { * signatures but do not extend a common parent. This is largely by design but * unfortunately leads to very similar test code in two places. */ -abstract class SessionCatalogSuite extends AnalysisTest { +abstract class SessionCatalogSuite extends AnalysisTest with Eventually { protected val utils: CatalogTestUtils protected val isHiveExternalCatalog = false @@ -70,6 +74,16 @@ abstract class SessionCatalogSuite extends AnalysisTest { catalog.reset() } } + + private def withConfAndEmptyCatalog(conf: SQLConf)(f: SessionCatalog => Unit): Unit = { + val catalog = new SessionCatalog(newEmptyCatalog(), new SimpleFunctionRegistry(), conf) + catalog.createDatabase(newDb("default"), ignoreIfExists = true) + try { + f(catalog) + } finally { + catalog.reset() + } + } // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -1641,4 +1655,27 @@ abstract class SessionCatalogSuite extends AnalysisTest { assert(cause.cause.get.getMessage.contains("Actual error")) } } + + test("expire table relation cache if TTL is configured") { + case class TestCommand() extends Command + + val conf = new SQLConf() + conf.setConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS, 1L) + + withConfAndEmptyCatalog(conf) { catalog => + val table = QualifiedTableName(catalog.getCurrentDatabase, "test") + + // First, make sure the test table is not cached. + assert(catalog.getCachedTable(table) === null) + + catalog.cacheTable(table, TestCommand()) + assert(catalog.getCachedTable(table) !== null) + + // Wait until the cache expiration. + eventually(timeout(3.seconds)) { + // And the cache is gone. + assert(catalog.getCachedTable(table) === null) + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index fd24f058f357c..d20a9ba3f0f68 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -336,7 +336,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { val encoder = RowEncoder(schema).resolveAndBind() val localDate = java.time.LocalDate.parse("2019-02-27") val row = toRow(encoder, Row(localDate)) - assert(row.getLong(0) === DateTimeUtils.localDateToDays(localDate)) + assert(row.getInt(0) === DateTimeUtils.localDateToDays(localDate)) val readback = fromRow(encoder, row) assert(readback.get(0).equals(localDate)) } 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 4ab288a34cb08..0e817a7652473 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 @@ -691,16 +691,22 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ret1, "[1, 2, 3, 4, 5]") val ret2 = cast(Literal.create(Array("ab", "cde", "f")), StringType) checkEvaluation(ret2, "[ab, cde, f]") - val ret3 = cast(Literal.create(Array("ab", null, "c")), StringType) - checkEvaluation(ret3, "[ab,, c]") - val ret4 = cast(Literal.create(Array("ab".getBytes, "cde".getBytes, "f".getBytes)), StringType) + Seq(false, true).foreach { omitNull => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> omitNull.toString) { + val ret3 = cast(Literal.create(Array("ab", null, "c")), StringType) + checkEvaluation(ret3, s"[ab,${if (omitNull) "" else " null"}, c]") + } + } + val ret4 = + cast(Literal.create(Array("ab".getBytes, "cde".getBytes, "f".getBytes)), StringType) checkEvaluation(ret4, "[ab, cde, f]") val ret5 = cast( Literal.create(Array("2014-12-03", "2014-12-04", "2014-12-06").map(Date.valueOf)), StringType) checkEvaluation(ret5, "[2014-12-03, 2014-12-04, 2014-12-06]") val ret6 = cast( - Literal.create(Array("2014-12-03 13:01:00", "2014-12-04 15:05:00").map(Timestamp.valueOf)), + Literal.create(Array("2014-12-03 13:01:00", "2014-12-04 15:05:00") + .map(Timestamp.valueOf)), StringType) checkEvaluation(ret6, "[2014-12-03 13:01:00, 2014-12-04 15:05:00]") val ret7 = cast(Literal.create(Array(Array(1, 2, 3), Array(4, 5))), StringType) @@ -712,47 +718,59 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-22973 Cast map to string") { - val ret1 = cast(Literal.create(Map(1 -> "a", 2 -> "b", 3 -> "c")), StringType) - checkEvaluation(ret1, "[1 -> a, 2 -> b, 3 -> c]") - val ret2 = cast( - Literal.create(Map("1" -> "a".getBytes, "2" -> null, "3" -> "c".getBytes)), - StringType) - checkEvaluation(ret2, "[1 -> a, 2 ->, 3 -> c]") - val ret3 = cast( - Literal.create(Map( - 1 -> Date.valueOf("2014-12-03"), - 2 -> Date.valueOf("2014-12-04"), - 3 -> Date.valueOf("2014-12-05"))), - StringType) - checkEvaluation(ret3, "[1 -> 2014-12-03, 2 -> 2014-12-04, 3 -> 2014-12-05]") - val ret4 = cast( - Literal.create(Map( - 1 -> Timestamp.valueOf("2014-12-03 13:01:00"), - 2 -> Timestamp.valueOf("2014-12-04 15:05:00"))), - StringType) - checkEvaluation(ret4, "[1 -> 2014-12-03 13:01:00, 2 -> 2014-12-04 15:05:00]") - val ret5 = cast( - Literal.create(Map( - 1 -> Array(1, 2, 3), - 2 -> Array(4, 5, 6))), - StringType) - checkEvaluation(ret5, "[1 -> [1, 2, 3], 2 -> [4, 5, 6]]") + Seq( + false -> ("{", "}"), + true -> ("[", "]")).foreach { case (legacyCast, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyCast.toString) { + val ret1 = cast(Literal.create(Map(1 -> "a", 2 -> "b", 3 -> "c")), StringType) + checkEvaluation(ret1, s"${lb}1 -> a, 2 -> b, 3 -> c$rb") + val ret2 = cast( + Literal.create(Map("1" -> "a".getBytes, "2" -> null, "3" -> "c".getBytes)), + StringType) + checkEvaluation(ret2, s"${lb}1 -> a, 2 ->${if (legacyCast) "" else " null"}, 3 -> c$rb") + val ret3 = cast( + Literal.create(Map( + 1 -> Date.valueOf("2014-12-03"), + 2 -> Date.valueOf("2014-12-04"), + 3 -> Date.valueOf("2014-12-05"))), + StringType) + checkEvaluation(ret3, s"${lb}1 -> 2014-12-03, 2 -> 2014-12-04, 3 -> 2014-12-05$rb") + val ret4 = cast( + Literal.create(Map( + 1 -> Timestamp.valueOf("2014-12-03 13:01:00"), + 2 -> Timestamp.valueOf("2014-12-04 15:05:00"))), + StringType) + checkEvaluation(ret4, s"${lb}1 -> 2014-12-03 13:01:00, 2 -> 2014-12-04 15:05:00$rb") + val ret5 = cast( + Literal.create(Map( + 1 -> Array(1, 2, 3), + 2 -> Array(4, 5, 6))), + StringType) + checkEvaluation(ret5, s"${lb}1 -> [1, 2, 3], 2 -> [4, 5, 6]$rb") + } + } } test("SPARK-22981 Cast struct to string") { - val ret1 = cast(Literal.create((1, "a", 0.1)), StringType) - checkEvaluation(ret1, "[1, a, 0.1]") - val ret2 = cast(Literal.create(Tuple3[Int, String, String](1, null, "a")), StringType) - checkEvaluation(ret2, "[1,, a]") - val ret3 = cast(Literal.create( - (Date.valueOf("2014-12-03"), Timestamp.valueOf("2014-12-03 15:05:00"))), StringType) - checkEvaluation(ret3, "[2014-12-03, 2014-12-03 15:05:00]") - val ret4 = cast(Literal.create(((1, "a"), 5, 0.1)), StringType) - checkEvaluation(ret4, "[[1, a], 5, 0.1]") - val ret5 = cast(Literal.create((Seq(1, 2, 3), "a", 0.1)), StringType) - checkEvaluation(ret5, "[[1, 2, 3], a, 0.1]") - val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) - checkEvaluation(ret6, "[1, [1 -> a, 2 -> b, 3 -> c]]") + Seq( + false -> ("{", "}"), + true -> ("[", "]")).foreach { case (legacyCast, (lb, rb)) => + withSQLConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING.key -> legacyCast.toString) { + val ret1 = cast(Literal.create((1, "a", 0.1)), StringType) + checkEvaluation(ret1, s"${lb}1, a, 0.1$rb") + val ret2 = cast(Literal.create(Tuple3[Int, String, String](1, null, "a")), StringType) + checkEvaluation(ret2, s"${lb}1,${if (legacyCast) "" else " null"}, a$rb") + val ret3 = cast(Literal.create( + (Date.valueOf("2014-12-03"), Timestamp.valueOf("2014-12-03 15:05:00"))), StringType) + checkEvaluation(ret3, s"${lb}2014-12-03, 2014-12-03 15:05:00$rb") + val ret4 = cast(Literal.create(((1, "a"), 5, 0.1)), StringType) + checkEvaluation(ret4, s"$lb${lb}1, a$rb, 5, 0.1$rb") + val ret5 = cast(Literal.create((Seq(1, 2, 3), "a", 0.1)), StringType) + checkEvaluation(ret5, s"$lb[1, 2, 3], a, 0.1$rb") + val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) + checkEvaluation(ret6, s"${lb}1, ${lb}1 -> a, 2 -> b, 3 -> c$rb$rb") + } + } } test("up-cast") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala deleted file mode 100644 index 793abccd79405..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConjunctiveNormalFormPredicateSuite.scala +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.BooleanType - -class ConjunctiveNormalFormPredicateSuite extends SparkFunSuite with PredicateHelper with PlanTest { - private val a = AttributeReference("A", BooleanType)(exprId = ExprId(1)).withQualifier(Seq("ta")) - private val b = AttributeReference("B", BooleanType)(exprId = ExprId(2)).withQualifier(Seq("tb")) - private val c = AttributeReference("C", BooleanType)(exprId = ExprId(3)).withQualifier(Seq("tc")) - private val d = AttributeReference("D", BooleanType)(exprId = ExprId(4)).withQualifier(Seq("td")) - private val e = AttributeReference("E", BooleanType)(exprId = ExprId(5)).withQualifier(Seq("te")) - private val f = AttributeReference("F", BooleanType)(exprId = ExprId(6)).withQualifier(Seq("tf")) - private val g = AttributeReference("G", BooleanType)(exprId = ExprId(7)).withQualifier(Seq("tg")) - private val h = AttributeReference("H", BooleanType)(exprId = ExprId(8)).withQualifier(Seq("th")) - private val i = AttributeReference("I", BooleanType)(exprId = ExprId(9)).withQualifier(Seq("ti")) - private val j = AttributeReference("J", BooleanType)(exprId = ExprId(10)).withQualifier(Seq("tj")) - private val a1 = - AttributeReference("a1", BooleanType)(exprId = ExprId(11)).withQualifier(Seq("ta")) - private val a2 = - AttributeReference("a2", BooleanType)(exprId = ExprId(12)).withQualifier(Seq("ta")) - private val b1 = - AttributeReference("b1", BooleanType)(exprId = ExprId(12)).withQualifier(Seq("tb")) - - // Check CNF conversion with expected expression, assuming the input has non-empty result. - private def checkCondition(input: Expression, expected: Expression): Unit = { - val cnf = CNFWithGroupExpressionsByQualifier(input) - assert(cnf.nonEmpty) - val result = cnf.reduceLeft(And) - assert(result.semanticEquals(expected)) - } - - test("Keep non-predicated expressions") { - checkCondition(a, a) - checkCondition(Literal(1), Literal(1)) - } - - test("Conversion of Not") { - checkCondition(!a, !a) - checkCondition(!(!a), a) - checkCondition(!(!(a && b)), a && b) - checkCondition(!(!(a || b)), a || b) - checkCondition(!(a || b), !a && !b) - checkCondition(!(a && b), !a || !b) - } - - test("Conversion of And") { - checkCondition(a && b, a && b) - checkCondition(a && b && c, a && b && c) - checkCondition(a && (b || c), a && (b || c)) - checkCondition((a || b) && c, (a || b) && c) - checkCondition(a && b && c && d, a && b && c && d) - } - - test("Conversion of Or") { - checkCondition(a || b, a || b) - checkCondition(a || b || c, a || b || c) - checkCondition(a || b || c || d, a || b || c || d) - checkCondition((a && b) || c, (a || c) && (b || c)) - checkCondition((a && b) || (c && d), (a || c) && (a || d) && (b || c) && (b || d)) - } - - test("More complex cases") { - checkCondition(a && !(b || c), a && !b && !c) - checkCondition((a && b) || !(c && d), (a || !c || !d) && (b || !c || !d)) - checkCondition(a || b || c && d, (a || b || c) && (a || b || d)) - checkCondition(a || (b && c || d), (a || b || d) && (a || c || d)) - checkCondition(a && !(b && c || d && e), a && (!b || !c) && (!d || !e)) - checkCondition(((a && b) || c) || (d || e), (a || c || d || e) && (b || c || d || e)) - - checkCondition( - (a && b && c) || (d && e && f), - (a || d) && (a || e) && (a || f) && (b || d) && (b || e) && (b || f) && - (c || d) && (c || e) && (c || f) - ) - } - - test("Aggregate predicate of same qualifiers to avoid expanding") { - checkCondition(((a && b && a1) || c), ((a && a1) || c) && (b ||c)) - checkCondition(((a && a1 && b) || c), ((a && a1) || c) && (b ||c)) - checkCondition(((b && d && a && a1) || c), ((a && a1) || c) && (b ||c) && (d || c)) - checkCondition(((b && a2 && d && a && a1) || c), ((a2 && a && a1) || c) && (b ||c) && (d || c)) - checkCondition(((b && d && a && a1 && b1) || c), - ((a && a1) || c) && ((b && b1) ||c) && (d || c)) - checkCondition((a && a1) || (b && b1), (a && a1) || (b && b1)) - checkCondition((a && a1 && c) || (b && b1), ((a && a1) || (b && b1)) && (c || (b && b1))) - } - - test("Return Seq.empty when exceeding MAX_CNF_NODE_COUNT") { - // The following expression contains 36 conjunctive sub-expressions in CNF - val input = (a && b && c) || (d && e && f) || (g && h && i && j) - // The following expression contains 9 conjunctive sub-expressions in CNF - val input2 = (a && b && c) || (d && e && f) - Seq(8, 9, 10, 35, 36, 37).foreach { maxCount => - withSQLConf(SQLConf.MAX_CNF_NODE_COUNT.key -> maxCount.toString) { - if (maxCount < 36) { - assert(CNFWithGroupExpressionsByQualifier(input).isEmpty) - } else { - assert(CNFWithGroupExpressionsByQualifier(input).nonEmpty) - } - if (maxCount < 9) { - assert(CNFWithGroupExpressionsByQualifier(input2).isEmpty) - } else { - assert(CNFWithGroupExpressionsByQualifier(input2).nonEmpty) - } - } - } - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExtractPredicatesWithinOutputSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExtractPredicatesWithinOutputSetSuite.scala new file mode 100644 index 0000000000000..ed141ef923e0a --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExtractPredicatesWithinOutputSetSuite.scala @@ -0,0 +1,82 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.types.BooleanType + +class ExtractPredicatesWithinOutputSetSuite + extends SparkFunSuite + with PredicateHelper + with PlanTest { + private val a = AttributeReference("A", BooleanType)(exprId = ExprId(1)) + private val b = AttributeReference("B", BooleanType)(exprId = ExprId(2)) + private val c = AttributeReference("C", BooleanType)(exprId = ExprId(3)) + private val d = AttributeReference("D", BooleanType)(exprId = ExprId(4)) + private val e = AttributeReference("E", BooleanType)(exprId = ExprId(5)) + private val f = AttributeReference("F", BooleanType)(exprId = ExprId(6)) + private val g = AttributeReference("G", BooleanType)(exprId = ExprId(7)) + private val h = AttributeReference("H", BooleanType)(exprId = ExprId(8)) + private val i = AttributeReference("I", BooleanType)(exprId = ExprId(9)) + + private def checkCondition( + input: Expression, + convertibleAttributes: Seq[Attribute], + expected: Option[Expression]): Unit = { + val result = extractPredicatesWithinOutputSet(input, AttributeSet(convertibleAttributes)) + if (expected.isEmpty) { + assert(result.isEmpty) + } else { + assert(result.isDefined && result.get.semanticEquals(expected.get)) + } + } + + test("Convertible conjunctive predicates") { + checkCondition(a && b, Seq(a, b), Some(a && b)) + checkCondition(a && b, Seq(a), Some(a)) + checkCondition(a && b, Seq(b), Some(b)) + checkCondition(a && b && c, Seq(a, c), Some(a && c)) + checkCondition(a && b && c && d, Seq(b, c), Some(b && c)) + } + + test("Convertible disjunctive predicates") { + checkCondition(a || b, Seq(a, b), Some(a || b)) + checkCondition(a || b, Seq(a), None) + checkCondition(a || b, Seq(b), None) + checkCondition(a || b || c, Seq(a, c), None) + checkCondition(a || b || c || d, Seq(a, b, d), None) + checkCondition(a || b || c || d, Seq(d, c, b, a), Some(a || b || c || d)) + } + + test("Convertible complex predicates") { + checkCondition((a && b) || (c && d), Seq(a, c), Some(a || c)) + checkCondition((a && b) || (c && d), Seq(a, b), None) + checkCondition((a && b) || (c && d), Seq(a, c, d), Some(a || (c && d))) + checkCondition((a && b && c) || (d && e && f), Seq(a, c, d, f), Some((a && c) || (d && f))) + checkCondition((a && b) || (c && d) || (e && f) || (g && h), Seq(a, c, e, g), + Some(a || c || e || g)) + checkCondition((a && b) || (c && d) || (e && f) || (g && h), Seq(a, e, g), None) + checkCondition((a || b) || (c && d) || (e && f) || (g && h), Seq(a, c, e, g), None) + checkCondition((a || b) || (c && d) || (e && f) || (g && h), Seq(a, b, c, e, g), + Some(a || b || c || e || g)) + checkCondition((a && b && c) || (d && e && f) || (g && h && i), Seq(b, e, h), Some(b || e || h)) + checkCondition((a && b && c) || (d && e && f) || (g && h && i), Seq(b, e, d), None) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index 0fea84bb183e0..02c5c9ab89088 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import scala.collection.JavaConverters._ +import scala.collection.mutable.WrappedArray import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.Random @@ -330,6 +331,8 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(result.asInstanceOf[ArrayData].array.toSeq == expected) case l if classOf[java.util.List[_]].isAssignableFrom(l) => assert(result.asInstanceOf[java.util.List[_]].asScala == expected) + case a if classOf[WrappedArray[Int]].isAssignableFrom(a) => + assert(result == WrappedArray.make[Int](expected.toArray)) case s if classOf[Seq[_]].isAssignableFrom(s) => assert(result.asInstanceOf[Seq[_]] == expected) case s if classOf[scala.collection.Set[_]].isAssignableFrom(s) => @@ -337,7 +340,8 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - val customCollectionClasses = Seq(classOf[Seq[Int]], classOf[scala.collection.Set[Int]], + val customCollectionClasses = Seq(classOf[WrappedArray[Int]], + classOf[Seq[Int]], classOf[scala.collection.Set[Int]], classOf[java.util.List[Int]], classOf[java.util.AbstractList[Int]], classOf[java.util.AbstractSequentialList[Int]], classOf[java.util.Vector[Int]], classOf[java.util.Stack[Int]], null) @@ -357,6 +361,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { stack.add(3) Seq( + (Seq(1, 2, 3), ObjectType(classOf[WrappedArray[Int]])), (Seq(1, 2, 3), ObjectType(classOf[Seq[Int]])), (Array(1, 2, 3), ObjectType(classOf[Array[Int]])), (Seq(1, 2, 3), ObjectType(classOf[Object])), 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 ad9492a8d3ab1..205dc10efc8a8 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 @@ -303,6 +303,8 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row8 = create_row("100-200", "(\\d+)-(\\d+)", 3) val row9 = create_row("100-200", "(\\d+).*", 2) val row10 = create_row("100-200", "\\d+", 1) + val row11 = create_row("100-200", "(\\d+)-(\\d+)", -1) + val row12 = create_row("100-200", "\\d+", -1) checkExceptionInExpression[IllegalArgumentException]( expr, row8, "Regex group count is 2, but the specified group index is 3") @@ -310,12 +312,66 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { expr, row9, "Regex group count is 1, but the specified group index is 2") checkExceptionInExpression[IllegalArgumentException]( expr, row10, "Regex group count is 0, but the specified group index is 1") + checkExceptionInExpression[IllegalArgumentException]( + expr, row11, "The specified group index cannot be less than zero") + checkExceptionInExpression[IllegalArgumentException]( + expr, row12, "The specified group index cannot be less than zero") // Test escaping of arguments GenerateUnsafeProjection.generate( RegExpExtract(Literal("\"quote"), Literal("\"quote"), Literal(1)) :: Nil) } + test("RegexExtractAll") { + val row1 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 0) + val row2 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 1) + val row3 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 2) + val row4 = create_row("100-200,300-400,500-600", "(\\d+).*", 1) + val row5 = create_row("100-200,300-400,500-600", "([a-z])", 1) + val row6 = create_row(null, "([a-z])", 1) + val row7 = create_row("100-200,300-400,500-600", null, 1) + val row8 = create_row("100-200,300-400,500-600", "([a-z])", null) + + val s = 's.string.at(0) + val p = 'p.string.at(1) + val r = 'r.int.at(2) + + val expr = RegExpExtractAll(s, p, r) + checkEvaluation(expr, Seq("100-200", "300-400", "500-600"), row1) + checkEvaluation(expr, Seq("100", "300", "500"), row2) + checkEvaluation(expr, Seq("200", "400", "600"), row3) + checkEvaluation(expr, Seq("100"), row4) + checkEvaluation(expr, Seq(), row5) + checkEvaluation(expr, null, row6) + checkEvaluation(expr, null, row7) + checkEvaluation(expr, null, row8) + + val expr1 = new RegExpExtractAll(s, p) + checkEvaluation(expr1, Seq("100", "300", "500"), row2) + + val nonNullExpr = RegExpExtractAll(Literal("100-200,300-400,500-600"), + Literal("(\\d+)-(\\d+)"), Literal(1)) + checkEvaluation(nonNullExpr, Seq("100", "300", "500"), row2) + + // invalid group index + val row9 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", 3) + val row10 = create_row("100-200,300-400,500-600", "(\\d+).*", 2) + val row11 = create_row("100-200,300-400,500-600", "\\d+", 1) + val row12 = create_row("100-200,300-400,500-600", "(\\d+)-(\\d+)", -1) + val row13 = create_row("100-200,300-400,500-600", "\\d+", -1) + + checkExceptionInExpression[IllegalArgumentException]( + expr, row9, "Regex group count is 2, but the specified group index is 3") + checkExceptionInExpression[IllegalArgumentException]( + expr, row10, "Regex group count is 1, but the specified group index is 2") + checkExceptionInExpression[IllegalArgumentException]( + expr, row11, "Regex group count is 0, but the specified group index is 1") + checkExceptionInExpression[IllegalArgumentException]( + expr, row12, "The specified group index cannot be less than zero") + checkExceptionInExpression[IllegalArgumentException]( + expr, row13, "The specified group index cannot be less than zero") + } + test("SPLIT") { val s1 = 'a.string.at(0) val s2 = 'b.string.at(1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 0149f0deb8d2b..220728fcaa2f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.expressions import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala similarity index 84% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala index ba36bc074e154..bb6672e1046da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala @@ -17,14 +17,15 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Literal} import org.apache.spark.sql.types.IntegerType -class LastTestSuite extends SparkFunSuite { +class FirstLastTestSuite extends SparkFunSuite { val input = AttributeReference("input", IntegerType, nullable = true)() - val evaluator = DeclarativeAggregateEvaluator(Last(input, Literal(false)), Seq(input)) - val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(Last(input, Literal(true)), Seq(input)) + val evaluator = DeclarativeAggregateEvaluator(Last(input, false), Seq(input)) + val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(Last(input, true), Seq(input)) test("empty buffer") { assert(evaluator.initialize() === InternalRow(null, false)) @@ -106,4 +107,15 @@ class LastTestSuite extends SparkFunSuite { val m1 = evaluatorIgnoreNulls.merge(p1, p2) assert(evaluatorIgnoreNulls.eval(m1) === InternalRow(1)) } + + test("SPARK-32344: correct error handling for a type mismatch") { + val msg1 = intercept[AnalysisException] { + new First(input, Literal(1, IntegerType)) + }.getMessage + assert(msg1.contains("The second argument in first should be a boolean literal")) + val msg2 = intercept[AnalysisException] { + new Last(input, Literal(1, IntegerType)) + }.getMessage + assert(msg2.contains("The second argument in last should be a boolean literal")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala index d2862c8f41d1b..972a832255155 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import org.scalatest.{Assertions, BeforeAndAfterEach, Matchers} +import org.scalatest.{Assertions, BeforeAndAfterEach} +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkFunSuite, TestUtils} import org.apache.spark.deploy.SparkSubmitSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala new file mode 100644 index 0000000000000..587e22e787b87 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.json + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.{EqualTo, Filter, StringStartsWith} +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.UTF8String + +class JacksonParserSuite extends SparkFunSuite { + test("skipping rows using pushdown filters") { + def check( + input: String = """{"i":1, "s": "a"}""", + schema: StructType = StructType.fromDDL("i INTEGER"), + filters: Seq[Filter], + expected: Seq[InternalRow]): Unit = { + val options = new JSONOptions(Map.empty[String, String], "GMT", "") + val parser = new JacksonParser(schema, options, false, filters) + val createParser = CreateJacksonParser.string _ + val actual = parser.parse(input, createParser, UTF8String.fromString) + assert(actual === expected) + } + + check(filters = Seq(), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 2)), expected = Seq.empty) + check( + schema = StructType.fromDDL("s STRING"), + filters = Seq(StringStartsWith("s", "b")), + expected = Seq.empty) + check( + schema = StructType.fromDDL("i INTEGER, s STRING"), + filters = Seq(StringStartsWith("s", "a")), + expected = Seq(InternalRow(1, UTF8String.fromString("a")))) + check( + input = """{"i":1,"s": "a", "d": 3.14}""", + schema = StructType.fromDDL("i INTEGER, d DOUBLE"), + filters = Seq(EqualTo("d", 3.14)), + expected = Seq(InternalRow(1, 3.14))) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonFiltersSuite.scala new file mode 100644 index 0000000000000..82f5e71d67964 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonFiltersSuite.scala @@ -0,0 +1,28 @@ +/* + * 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.catalyst.json + +import org.apache.spark.sql.catalyst.{StructFilters, StructFiltersSuite} +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType + +class JsonFiltersSuite extends StructFiltersSuite { + override def createFilters(filters: Seq[sources.Filter], schema: StructType): StructFilters = { + new JsonFilters(filters, schema) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala index 788fedb3c8e8e..dea2b36ecc844 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala new file mode 100644 index 0000000000000..9f031358611b1 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala @@ -0,0 +1,193 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class EliminateSortsBeforeRepartitionSuite extends PlanTest { + + val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) + val analyzer = new Analyzer(catalog, conf) + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val anotherTestRelation = LocalRelation('d.int, 'e.int) + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Default", FixedPoint(10), + FoldablePropagation, + LimitPushDown) :: + Batch("Eliminate Sorts", Once, + EliminateSorts) :: + Batch("Collapse Project", Once, + CollapseProject) :: Nil + } + + def repartition(plan: LogicalPlan): LogicalPlan = plan.repartition(10) + + test("sortBy") { + val plan = testRelation.select('a, 'b).sortBy('a.asc, 'b.desc) + val optimizedPlan = testRelation.select('a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with projection") { + val plan = testRelation.sortBy('a.asc, 'b.asc).select('a + 1 as "a", 'b + 2 as "b") + val optimizedPlan = testRelation.select('a + 1 as "a", 'b + 2 as "b") + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with projection and filter") { + val plan = testRelation.sortBy('a.asc, 'b.asc).select('a, 'b).where('a === 10) + val optimizedPlan = testRelation.select('a, 'b).where('a === 10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with limit") { + val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + val optimizedPlan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("sortBy with non-deterministic projection") { + val plan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val optimizedPlan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy") { + val plan = testRelation.select('a, 'b).orderBy('a.asc, 'b.asc) + val optimizedPlan = testRelation.select('a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with projection") { + val plan = testRelation.orderBy('a.asc, 'b.asc).select('a + 1 as "a", 'b + 2 as "b") + val optimizedPlan = testRelation.select('a + 1 as "a", 'b + 2 as "b") + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with projection and filter") { + val plan = testRelation.orderBy('a.asc, 'b.asc).select('a, 'b).where('a === 10) + val optimizedPlan = testRelation.select('a, 'b).where('a === 10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with limit") { + val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + val optimizedPlan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("orderBy with non-deterministic projection") { + val plan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val optimizedPlan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + checkRepartitionCases(plan, optimizedPlan) + } + + test("additional coalesce and sortBy") { + val plan = testRelation.sortBy('a.asc, 'b.asc).coalesce(1) + val optimizedPlan = testRelation.coalesce(1) + checkRepartitionCases(plan, optimizedPlan) + } + + test("additional projection, repartition and sortBy") { + val plan = testRelation.sortBy('a.asc, 'b.asc).repartition(100).select('a + 1 as "a") + val optimizedPlan = testRelation.repartition(100).select('a + 1 as "a") + checkRepartitionCases(plan, optimizedPlan) + } + + test("additional filter, distribute and sortBy") { + val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) + val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) + checkRepartitionCases(plan, optimizedPlan) + } + + test("join") { + val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) + val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) + val anotherPlan = anotherTestRelation.select('d) + val joinPlan = plan.join(anotherPlan) + val optimizedJoinPlan = optimize(joinPlan) + val correctJoinPlan = analyze(optimizedPlan.join(anotherPlan)) + comparePlans(optimizedJoinPlan, correctJoinPlan) + } + + test("aggregate") { + val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) + val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) + val aggPlan = plan.groupBy('a)(sum('b)) + val optimizedAggPlan = optimize(aggPlan) + val correctAggPlan = analyze(optimizedPlan.groupBy('a)(sum('b))) + comparePlans(optimizedAggPlan, correctAggPlan) + } + + protected def checkRepartitionCases(plan: LogicalPlan, optimizedPlan: LogicalPlan): Unit = { + // cannot remove sortBy before repartition without sortBy/orderBy + val planWithRepartition = repartition(plan) + val optimizedPlanWithRepartition = optimize(planWithRepartition) + val correctPlanWithRepartition = analyze(planWithRepartition) + comparePlans(optimizedPlanWithRepartition, correctPlanWithRepartition) + + // can remove sortBy before repartition with sortBy + val planWithRepartitionAndSortBy = planWithRepartition.sortBy('a.asc) + val optimizedPlanWithRepartitionAndSortBy = optimize(planWithRepartitionAndSortBy) + val correctPlanWithRepartitionAndSortBy = analyze(repartition(optimizedPlan).sortBy('a.asc)) + comparePlans(optimizedPlanWithRepartitionAndSortBy, correctPlanWithRepartitionAndSortBy) + + // can remove sortBy before repartition with orderBy + val planWithRepartitionAndOrderBy = planWithRepartition.orderBy('a.asc) + val optimizedPlanWithRepartitionAndOrderBy = optimize(planWithRepartitionAndOrderBy) + val correctPlanWithRepartitionAndOrderBy = analyze(repartition(optimizedPlan).orderBy('a.asc)) + comparePlans(optimizedPlanWithRepartitionAndOrderBy, correctPlanWithRepartitionAndOrderBy) + } + + private def analyze(plan: LogicalPlan): LogicalPlan = { + analyzer.execute(plan) + } + + private def optimize(plan: LogicalPlan): LogicalPlan = { + Optimize.execute(analyzer.execute(plan)) + } +} + +class EliminateSortsBeforeRepartitionByExprsSuite extends EliminateSortsBeforeRepartitionSuite { + override def repartition(plan: LogicalPlan): LogicalPlan = plan.distribute('a)(10) + + test("sortBy before repartition with non-deterministic expressions") { + val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + checkRepartitionCases(plan = planWithRepartition, optimizedPlan = planWithRepartition) + } + + test("orderBy before repartition with non-deterministic expressions") { + val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + checkRepartitionCases(plan = planWithRepartition, optimizedPlan = planWithRepartition) + } +} + +class EliminateSortsBeforeCoalesceSuite extends EliminateSortsBeforeRepartitionSuite { + override def repartition(plan: LogicalPlan): LogicalPlan = plan.coalesce(1) +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index bb7e9d04c12d9..cf92e25ccab48 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -33,9 +33,6 @@ class FilterPushdownSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - override protected val excludedOnceBatches: Set[String] = - Set("Push CNF predicate through join") - val batches = Batch("Subqueries", Once, EliminateSubqueryAliases) :: @@ -45,8 +42,9 @@ class FilterPushdownSuite extends PlanTest { BooleanSimplification, PushPredicateThroughJoin, CollapseProject) :: - Batch("Push CNF predicate through join", Once, - PushCNFPredicateThroughJoin) :: Nil + Batch("Push extra predicate through join", FixedPoint(10), + PushExtraPredicateThroughJoin, + PushDownPredicates) :: Nil } val attrA = 'a.int @@ -60,7 +58,7 @@ class FilterPushdownSuite extends PlanTest { val simpleDisjunctivePredicate = ("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11) - val expectedCNFPredicatePushDownResult = { + val expectedPredicatePushDownResult = { val left = testRelation.where(('a > 3 || 'a > 1)).subquery('x) val right = testRelation.where('a > 13 || 'a > 11).subquery('y) left.join(right, condition = Some("x.b".attr === "y.b".attr @@ -1247,17 +1245,17 @@ class FilterPushdownSuite extends PlanTest { comparePlans(Optimize.execute(query.analyze), expected) } - test("inner join: rewrite filter predicates to conjunctive normal form") { + test("push down filter predicates through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) val originalQuery = x.join(y).where(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate)) val optimized = Optimize.execute(originalQuery.analyze) - comparePlans(optimized, expectedCNFPredicatePushDownResult) + comparePlans(optimized, expectedPredicatePushDownResult) } - test("inner join: rewrite join predicates to conjunctive normal form") { + test("push down join predicates through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1265,10 +1263,10 @@ class FilterPushdownSuite extends PlanTest { x.join(y, condition = Some(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate))) val optimized = Optimize.execute(originalQuery.analyze) - comparePlans(optimized, expectedCNFPredicatePushDownResult) + comparePlans(optimized, expectedPredicatePushDownResult) } - test("inner join: rewrite complex join predicates to conjunctive normal form") { + test("push down complex predicates through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1288,7 +1286,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("inner join: rewrite join predicates(with NOT predicate) to conjunctive normal form") { + test("push down predicates(with NOT predicate) through inner join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1308,7 +1306,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("left join: rewrite join predicates to conjunctive normal form") { + test("push down predicates through left join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1327,7 +1325,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("right join: rewrite join predicates to conjunctive normal form") { + test("push down predicates through right join") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1346,7 +1344,7 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("inner join: rewrite to conjunctive normal form avoid generating too many predicates") { + test("SPARK-32302: avoid generating too many predicates") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -1364,30 +1362,20 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test(s"Disable rewrite to CNF by setting ${SQLConf.MAX_CNF_NODE_COUNT.key}=0") { + test("push down predicate through multiple joins") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) + val z = testRelation.subquery('z) + val xJoinY = x.join(y, condition = Some("x.b".attr === "y.b".attr)) + val originalQuery = z.join(xJoinY, + condition = Some("x.a".attr === "z.a".attr && simpleDisjunctivePredicate)) - val originalQuery = - x.join(y, condition = Some(("x.b".attr === "y.b".attr) - && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) - || (("y.a".attr > 2) && ("y.c".attr < 1))))) - - Seq(0, 10).foreach { count => - withSQLConf(SQLConf.MAX_CNF_NODE_COUNT.key -> count.toString) { - val optimized = Optimize.execute(originalQuery.analyze) - val (left, right) = if (count == 0) { - (testRelation.subquery('x), testRelation.subquery('y)) - } else { - (testRelation.subquery('x), - testRelation.where('c <= 5 || ('a > 2 && 'c < 1)).subquery('y)) - } - val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr - && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) - || (("y.a".attr > 2) && ("y.c".attr < 1))))).analyze - - comparePlans(optimized, correctAnswer) - } - } + val optimized = Optimize.execute(originalQuery.analyze) + val left = x.where('a > 3 || 'a > 1) + val right = y.where('a > 13 || 'a > 11) + val correctAnswer = z.join(left.join(right, + condition = Some("x.b".attr === "y.b".attr && simpleDisjunctivePredicate)), + condition = Some("x.a".attr === "z.a".attr)).analyze + comparePlans(optimized, correctAnswer) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index 0425846637f30..c83ab375ee15a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -145,9 +145,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val ops = Seq( (input: LogicalPlan) => input.distribute('name)(1), (input: LogicalPlan) => input.orderBy('name.asc), - (input: LogicalPlan) => input.orderBy($"name.middle".asc), (input: LogicalPlan) => input.sortBy('name.asc), - (input: LogicalPlan) => input.sortBy($"name.middle".asc), (input: LogicalPlan) => input.union(input) ) @@ -493,6 +491,144 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized3, expected3) } + test("Nested field pruning for Window") { + val spec = windowSpec($"address" :: Nil, $"id".asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(RowNumber(), spec) + val query = contact + .select($"name.first", winExpr.as('window)) + .orderBy($"name.last".asc) + .analyze + val optimized = Optimize.execute(query) + val aliases = collectGeneratedAliases(optimized) + val expected = contact + .select($"name.first", $"address", $"id", $"name.last".as(aliases(1))) + .window(Seq(winExpr.as("window")), Seq($"address"), Seq($"id".asc)) + .select($"first", $"window", $"${aliases(1)}".as(aliases(0))) + .orderBy($"${aliases(0)}".asc) + .select($"first", $"window") + .analyze + comparePlans(optimized, expected) + } + + test("Nested field pruning for Filter with other supported operators") { + val spec = windowSpec($"address" :: Nil, $"id".asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(RowNumber(), spec) + val query1 = contact.select($"name.first", winExpr.as('window)) + .where($"window" === 1 && $"name.first" === "a") + .analyze + val optimized1 = Optimize.execute(query1) + val aliases1 = collectGeneratedAliases(optimized1) + val expected1 = contact + .select($"name.first", $"address", $"id", $"name.first".as(aliases1(1))) + .window(Seq(winExpr.as("window")), Seq($"address"), Seq($"id".asc)) + .select($"first", $"${aliases1(1)}".as(aliases1(0)), $"window") + .where($"window" === 1 && $"${aliases1(0)}" === "a") + .select($"first", $"window") + .analyze + comparePlans(optimized1, expected1) + + val query2 = contact.sortBy($"name.first".asc) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized2 = Optimize.execute(query2) + val aliases2 = collectGeneratedAliases(optimized2) + val expected2 = contact + .select($"name.first".as(aliases2(1))) + .sortBy($"${aliases2(1)}".asc) + .select($"${aliases2(1)}".as(aliases2(0))) + .where($"${aliases2(0)}" === "a") + .select($"${aliases2(0)}".as("first")) + .analyze + comparePlans(optimized2, expected2) + + val query3 = contact.distribute($"name.first")(100) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized3 = Optimize.execute(query3) + val aliases3 = collectGeneratedAliases(optimized3) + val expected3 = contact + .select($"name.first".as(aliases3(1))) + .distribute($"${aliases3(1)}")(100) + .select($"${aliases3(1)}".as(aliases3(0))) + .where($"${aliases3(0)}" === "a") + .select($"${aliases3(0)}".as("first")) + .analyze + comparePlans(optimized3, expected3) + + val department = LocalRelation( + 'depID.int, + 'personID.string) + val query4 = contact.join(department, condition = Some($"id" === $"depID")) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized4 = Optimize.execute(query4) + val aliases4 = collectGeneratedAliases(optimized4) + val expected4 = contact + .select($"id", $"name.first".as(aliases4(1))) + .join(department.select('depID), condition = Some($"id" === $"depID")) + .select($"${aliases4(1)}".as(aliases4(0))) + .where($"${aliases4(0)}" === "a") + .select($"${aliases4(0)}".as("first")) + .analyze + comparePlans(optimized4, expected4) + + def runTest(basePlan: LogicalPlan => LogicalPlan): Unit = { + val query = basePlan(contact) + .where($"name.first" === "a") + .select($"name.first") + .analyze + val optimized = Optimize.execute(query) + val aliases = collectGeneratedAliases(optimized) + val expected = basePlan(contact + .select($"name.first".as(aliases(0)))) + .where($"${aliases(0)}" === "a") + .select($"${aliases(0)}".as("first")) + .analyze + comparePlans(optimized, expected) + } + Seq( + (plan: LogicalPlan) => plan.limit(100), + (plan: LogicalPlan) => plan.repartition(100), + (plan: LogicalPlan) => Sample(0.0, 0.6, false, 11L, plan)).foreach { base => + runTest(base) + } + } + + test("Nested field pruning for Sort") { + val query1 = contact.select($"name.first", $"name.last") + .sortBy($"name.first".asc, $"name.last".asc) + .analyze + val optimized1 = Optimize.execute(query1) + val aliases1 = collectGeneratedAliases(optimized1) + val expected1 = contact + .select($"name.first", + $"name.last", + $"name.first".as(aliases1(0)), + $"name.last".as(aliases1(1))) + .sortBy($"${aliases1(0)}".asc, $"${aliases1(1)}".asc) + .select($"first", $"last") + .analyze + comparePlans(optimized1, expected1) + + val query2 = contact.select($"name.first", $"name.last") + .orderBy($"name.first".asc, $"name.last".asc) + .analyze + val optimized2 = Optimize.execute(query2) + val aliases2 = collectGeneratedAliases(optimized2) + val expected2 = contact + .select($"name.first", + $"name.last", + $"name.first".as(aliases2(0)), + $"name.last".as(aliases2(1))) + .orderBy($"${aliases2(0)}".asc, $"${aliases2(1)}".asc) + .select($"first", $"last") + .analyze + comparePlans(optimized2, expected2) + } + test("Nested field pruning for Expand") { def runTest(basePlan: LogicalPlan => LogicalPlan): Unit = { val query1 = Expand( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala index ccc30b1d2f8ce..2eea840e21a31 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala @@ -223,4 +223,21 @@ class SetOperationSuite extends PlanTest { val unionCorrectAnswer = unionQuery.analyze comparePlans(unionOptimized, unionCorrectAnswer) } + + test("CombineUnions only flatten the unions with same byName and allowMissingCol") { + val union1 = Union(testRelation :: testRelation :: Nil, true, false) + val union2 = Union(testRelation :: testRelation :: Nil, true, true) + val union3 = Union(testRelation :: testRelation2 :: Nil, false, false) + + val union4 = Union(union1 :: union2 :: union3 :: Nil) + val unionOptimized1 = Optimize.execute(union4) + val unionCorrectAnswer1 = Union(union1 :: union2 :: testRelation :: testRelation2 :: Nil) + comparePlans(unionOptimized1, unionCorrectAnswer1, false) + + val union5 = Union(union1 :: union1 :: Nil, true, false) + val unionOptimized2 = Optimize.execute(union5) + val unionCorrectAnswer2 = + Union(testRelation :: testRelation :: testRelation :: testRelation :: Nil, true, false) + comparePlans(unionOptimized2, unionCorrectAnswer2, false) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index e802449a69743..ac6af4f4e3231 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedFunc, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView} import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, BucketSpec, FileResource, FunctionResource, FunctionResourceType, JarResource} import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ @@ -1962,7 +1962,6 @@ class DDLParserSuite extends AnalysisTest { """ |CREATE OR REPLACE GLOBAL TEMPORARY VIEW a.b.c |(col1, col3 COMMENT 'hello') - |TBLPROPERTIES('prop1Key'="prop1Val") |COMMENT 'BLABLA' |AS SELECT * FROM tab1 """.stripMargin @@ -1971,7 +1970,7 @@ class DDLParserSuite extends AnalysisTest { Seq("a", "b", "c"), Seq("col1" -> None, "col3" -> Some("hello")), Some("BLABLA"), - Map("prop1Key" -> "prop1Val"), + Map(), Some("SELECT * FROM tab1"), parsePlan("SELECT * FROM tab1"), false, @@ -2003,6 +2002,17 @@ class DDLParserSuite extends AnalysisTest { intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") } + test("SPARK-32374: create temporary view with properties not allowed") { + assertUnsupported( + sql = """ + |CREATE OR REPLACE TEMPORARY VIEW a.b.c + |(col1, col3 COMMENT 'hello') + |TBLPROPERTIES('prop1Key'="prop1Val") + |AS SELECT * FROM tab1 + """.stripMargin, + containsThesePhrases = Seq("TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW")) + } + test("SHOW TBLPROPERTIES table") { comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c"), @@ -2016,40 +2026,40 @@ class DDLParserSuite extends AnalysisTest { test("DESCRIBE FUNCTION") { comparePlans( parsePlan("DESC FUNCTION a"), - DescribeFunctionStatement(Seq("a"), false)) + DescribeFunction(UnresolvedFunc(Seq("a")), false)) comparePlans( parsePlan("DESCRIBE FUNCTION a"), - DescribeFunctionStatement(Seq("a"), false)) + DescribeFunction(UnresolvedFunc(Seq("a")), false)) comparePlans( parsePlan("DESCRIBE FUNCTION a.b.c"), - DescribeFunctionStatement(Seq("a", "b", "c"), false)) + DescribeFunction(UnresolvedFunc(Seq("a", "b", "c")), false)) comparePlans( parsePlan("DESCRIBE FUNCTION EXTENDED a.b.c"), - DescribeFunctionStatement(Seq("a", "b", "c"), true)) + DescribeFunction(UnresolvedFunc(Seq("a", "b", "c")), true)) } test("SHOW FUNCTIONS") { comparePlans( parsePlan("SHOW FUNCTIONS"), - ShowFunctionsStatement(true, true, None, None)) + ShowFunctions(None, true, true, None)) comparePlans( parsePlan("SHOW USER FUNCTIONS"), - ShowFunctionsStatement(true, false, None, None)) + ShowFunctions(None, true, false, None)) comparePlans( parsePlan("SHOW user FUNCTIONS"), - ShowFunctionsStatement(true, false, None, None)) + ShowFunctions(None, true, false, None)) comparePlans( parsePlan("SHOW SYSTEM FUNCTIONS"), - ShowFunctionsStatement(false, true, None, None)) + ShowFunctions(None, false, true, None)) comparePlans( parsePlan("SHOW ALL FUNCTIONS"), - ShowFunctionsStatement(true, true, None, None)) + ShowFunctions(None, true, true, None)) comparePlans( parsePlan("SHOW FUNCTIONS LIKE 'funct*'"), - ShowFunctionsStatement(true, true, Some("funct*"), None)) + ShowFunctions(None, true, true, Some("funct*"))) comparePlans( parsePlan("SHOW FUNCTIONS LIKE a.b.c"), - ShowFunctionsStatement(true, true, None, Some(Seq("a", "b", "c")))) + ShowFunctions(Some(UnresolvedFunc(Seq("a", "b", "c"))), true, true, None)) val sql = "SHOW other FUNCTIONS" intercept(sql, s"$sql not supported") } @@ -2057,19 +2067,19 @@ class DDLParserSuite extends AnalysisTest { test("DROP FUNCTION") { comparePlans( parsePlan("DROP FUNCTION a"), - DropFunctionStatement(Seq("a"), false, false)) + DropFunction(UnresolvedFunc(Seq("a")), false, false)) comparePlans( parsePlan("DROP FUNCTION a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), false, false)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), false, false)) comparePlans( parsePlan("DROP TEMPORARY FUNCTION a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), false, true)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), false, true)) comparePlans( parsePlan("DROP FUNCTION IF EXISTS a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), true, false)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), true, false)) comparePlans( parsePlan("DROP TEMPORARY FUNCTION IF EXISTS a.b.c"), - DropFunctionStatement(Seq("a", "b", "c"), true, true)) + DropFunction(UnresolvedFunc(Seq("a", "b", "c")), true, true)) } test("CREATE FUNCTION") { @@ -2109,6 +2119,15 @@ class DDLParserSuite extends AnalysisTest { "Operation not allowed: CREATE FUNCTION with resource type 'other'") } + test("REFRESH FUNCTION") { + parseCompare("REFRESH FUNCTION c", + RefreshFunction(UnresolvedFunc(Seq("c")))) + parseCompare("REFRESH FUNCTION b.c", + RefreshFunction(UnresolvedFunc(Seq("b", "c")))) + parseCompare("REFRESH FUNCTION a.b.c", + RefreshFunction(UnresolvedFunc(Seq("a", "b", "c")))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 41652cab2675b..bcbdf5df57d0e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -785,10 +785,10 @@ class ExpressionParserSuite extends AnalysisTest { } test("SPARK-19526 Support ignore nulls keywords for first and last") { - assertEqual("first(a ignore nulls)", First('a, Literal(true)).toAggregateExpression()) - assertEqual("first(a)", First('a, Literal(false)).toAggregateExpression()) - assertEqual("last(a ignore nulls)", Last('a, Literal(true)).toAggregateExpression()) - assertEqual("last(a)", Last('a, Literal(false)).toAggregateExpression()) + assertEqual("first(a ignore nulls)", First('a, true).toAggregateExpression()) + assertEqual("first(a)", First('a, false).toAggregateExpression()) + assertEqual("last(a ignore nulls)", Last('a, true).toAggregateExpression()) + assertEqual("last(a)", Last('a, false).toAggregateExpression()) } test("timestamp literals") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index f5259706325eb..ff51bc0071c80 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -583,7 +583,9 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { JObject( "class" -> classOf[Union].getName, "num-children" -> 2, - "children" -> List(0, 1)), + "children" -> List(0, 1), + "byName" -> JBool(false), + "allowMissingCol" -> JBool(false)), JObject( "class" -> classOf[JsonTestTreeNode].getName, "num-children" -> 0, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala index a8bb1d0afdb87..db8b7403cb60e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMapSuite.scala @@ -44,4 +44,12 @@ class CaseInsensitiveMapSuite extends SparkFunSuite { assert(m == Map("a" -> "b", "foo" -> "bar", "x" -> "y")) shouldBeSerializable(m) } + + test("SPARK-32377: CaseInsensitiveMap should be deterministic for addition") { + var m = CaseInsensitiveMap(Map.empty[String, String]) + Seq(("paTh", "1"), ("PATH", "2"), ("Path", "3"), ("patH", "4"), ("path", "5")).foreach { kv => + m = (m + kv).asInstanceOf[CaseInsensitiveMap[String]] + assert(m.get("path").contains(kv._2)) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index caf4b7e16f285..fe761f672c041 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -23,7 +23,8 @@ import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId} import java.util.Locale import java.util.concurrent.TimeUnit -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala index 3b9a4ae88d586..54a0aec4cf612 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.time.DateTimeException -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkFunSuite, SparkUpgradeException} import org.apache.spark.sql.catalyst.plans.SQLHelper diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index b74a219540bb2..428a0c028a5f6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -21,7 +21,7 @@ import java.sql.{Date, Timestamp} import java.time.{Instant, LocalDate, LocalDateTime, ZoneId} import java.util.TimeZone -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index 79c06cf8313b8..103b7a2eded28 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.util import java.time.{DateTimeException, Instant, LocalDateTime, LocalTime} import java.util.concurrent.TimeUnit +import org.scalatest.matchers.should.Matchers._ + import org.apache.spark.SparkUpgradeException import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ @@ -435,4 +437,14 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { assert(formatter.format(date(1970, 4, 10)) == "100") } } + + test("SPARK-32424: avoid silent data change when timestamp overflows") { + val formatter = TimestampFormatter("y", UTC, isParsing = true) + assert(formatter.parse("294247") === date(294247)) + assert(formatter.parse("-290307") === date(-290307)) + val e1 = intercept[ArithmeticException](formatter.parse("294248")) + assert(e1.getMessage === "long overflow") + val e2 = intercept[ArithmeticException](formatter.parse("-290308")) + assert(e2.getMessage === "long overflow") + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala index ebc88612be22a..443534fd0a06a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala @@ -48,8 +48,8 @@ class UnsafeMapSuite extends SparkFunSuite { val ser = new JavaSerializer(new SparkConf).newInstance() val mapDataSer = ser.deserialize[UnsafeMapData](ser.serialize(unsafeMapData)) assert(mapDataSer.numElements() == 1) - assert(mapDataSer.keyArray().getInt(0) == 19285) - assert(mapDataSer.valueArray().getInt(0) == 19286) + assert(mapDataSer.keyArray().getLong(0) == 19285) + assert(mapDataSer.valueArray().getLong(0) == 19286) assert(mapDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) } @@ -57,8 +57,8 @@ class UnsafeMapSuite extends SparkFunSuite { val ser = new KryoSerializer(new SparkConf).newInstance() val mapDataSer = ser.deserialize[UnsafeMapData](ser.serialize(unsafeMapData)) assert(mapDataSer.numElements() == 1) - assert(mapDataSer.keyArray().getInt(0) == 19285) - assert(mapDataSer.valueArray().getInt(0) == 19286) + assert(mapDataSer.keyArray().getLong(0) == 19285) + assert(mapDataSer.valueArray().getLong(0) == 19286) assert(mapDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index b36ded3d9d2ac..0db758d5147f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -20,7 +20,7 @@ import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.scalatest.Inside -import org.scalatest.Matchers._ +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index 2f576a4031e92..02ee634dba1b6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{ArrayType, LongType, MapType, StructType} class SchemaUtilsSuite extends SparkFunSuite { @@ -82,4 +82,28 @@ class SchemaUtilsSuite extends SparkFunSuite { checkNoExceptionCases("a INT, b INT, c INT", caseSensitive = false) } + + test("SPARK-32431: duplicated fields in nested schemas") { + val schemaA = new StructType() + .add("LowerCase", LongType) + .add("camelcase", LongType) + .add("CamelCase", LongType) + val schemaB = new StructType() + .add("f1", LongType) + .add("StructColumn1", schemaA) + val schemaC = new StructType() + .add("f2", LongType) + .add("StructColumn2", schemaB) + val schemaD = new StructType() + .add("f3", ArrayType(schemaC)) + val schemaE = MapType(LongType, schemaD) + val schemaF = MapType(schemaD, LongType) + Seq(schemaA, schemaB, schemaC, schemaD, schemaE, schemaF).foreach { schema => + val msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = false) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `camelcase`")) + } + } } diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index 0e82b632793d2..03c51ddad1c62 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -6,62 +6,62 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 46568 46683 198 0.0 931358.6 1.0X +One quoted string 53332 53484 194 0.0 1066633.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 129836 130796 1404 0.0 129836.0 1.0X -Select 100 columns 40444 40679 261 0.0 40443.5 3.2X -Select one column 33429 33475 73 0.0 33428.6 3.9X -count() 7967 8047 73 0.1 7966.7 16.3X -Select 100 columns, one bad input field 90639 90832 266 0.0 90638.6 1.4X -Select 100 columns, corrupt record field 109023 109084 74 0.0 109023.3 1.2X +Select 1000 columns 127472 128337 1185 0.0 127472.4 1.0X +Select 100 columns 43731 43856 130 0.0 43730.7 2.9X +Select one column 37347 37401 47 0.0 37347.4 3.4X +count() 8014 8028 25 0.1 8013.8 15.9X +Select 100 columns, one bad input field 95603 95726 106 0.0 95603.0 1.3X +Select 100 columns, corrupt record field 111851 111969 171 0.0 111851.4 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 20685 20707 35 0.5 2068.5 1.0X -Select 1 column + count() 13096 13149 49 0.8 1309.6 1.6X -count() 3994 4001 7 2.5 399.4 5.2X +Select 10 columns + count() 20364 20481 110 0.5 2036.4 1.0X +Select 1 column + count() 14706 14803 153 0.7 1470.6 1.4X +count() 3855 3880 32 2.6 385.5 5.3X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2169 2203 32 4.6 216.9 1.0X -to_csv(timestamp) 14401 14591 168 0.7 1440.1 0.2X -write timestamps to files 13209 13276 59 0.8 1320.9 0.2X -Create a dataset of dates 2231 2248 17 4.5 223.1 1.0X -to_csv(date) 10406 10473 68 1.0 1040.6 0.2X -write dates to files 7970 7976 9 1.3 797.0 0.3X +Create a dataset of timestamps 2191 2205 14 4.6 219.1 1.0X +to_csv(timestamp) 13209 13253 43 0.8 1320.9 0.2X +write timestamps to files 12300 12380 71 0.8 1230.0 0.2X +Create a dataset of dates 2254 2269 14 4.4 225.4 1.0X +to_csv(date) 7980 8006 22 1.3 798.0 0.3X +write dates to files 7076 7100 26 1.4 707.6 0.3X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2387 2391 6 4.2 238.7 1.0X -read timestamps from files 53503 53593 124 0.2 5350.3 0.0X -infer timestamps from files 107988 108668 647 0.1 10798.8 0.0X -read date text from files 2121 2133 12 4.7 212.1 1.1X -read date from files 29983 30039 48 0.3 2998.3 0.1X -infer date from files 30196 30436 218 0.3 3019.6 0.1X -timestamp strings 3098 3109 10 3.2 309.8 0.8X -parse timestamps from Dataset[String] 63331 63426 84 0.2 6333.1 0.0X -infer timestamps from Dataset[String] 124003 124463 490 0.1 12400.3 0.0X -date strings 3423 3429 11 2.9 342.3 0.7X -parse dates from Dataset[String] 34235 34314 76 0.3 3423.5 0.1X -from_csv(timestamp) 60829 61600 668 0.2 6082.9 0.0X -from_csv(date) 33047 33173 139 0.3 3304.7 0.1X +read timestamp text from files 2405 2408 5 4.2 240.5 1.0X +read timestamps from files 54502 54624 207 0.2 5450.2 0.0X +infer timestamps from files 112896 113040 135 0.1 11289.6 0.0X +read date text from files 2127 2141 23 4.7 212.7 1.1X +read date from files 30229 30257 29 0.3 3022.9 0.1X +infer date from files 28156 28621 409 0.4 2815.6 0.1X +timestamp strings 3096 3097 1 3.2 309.6 0.8X +parse timestamps from Dataset[String] 63096 63751 571 0.2 6309.6 0.0X +infer timestamps from Dataset[String] 120916 121262 556 0.1 12091.6 0.0X +date strings 3445 3457 13 2.9 344.5 0.7X +parse dates from Dataset[String] 37481 37585 91 0.3 3748.1 0.1X +from_csv(timestamp) 57933 57996 69 0.2 5793.3 0.0X +from_csv(date) 35312 35469 164 0.3 3531.2 0.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 28752 28765 16 0.0 287516.5 1.0X -pushdown disabled 28856 28880 22 0.0 288556.3 1.0X -w/ filters 1714 1731 15 0.1 17137.3 16.8X +w/o filters 24751 24829 67 0.0 247510.6 1.0X +pushdown disabled 24856 24889 29 0.0 248558.7 1.0X +w/ filters 1881 1892 11 0.1 18814.4 13.2X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index a3af46c037bf9..a0d8c0c6fd492 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -6,62 +6,62 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 45457 45731 344 0.0 909136.8 1.0X +One quoted string 47588 47831 244 0.0 951755.4 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 129646 130527 1412 0.0 129646.3 1.0X -Select 100 columns 42444 42551 119 0.0 42444.0 3.1X -Select one column 35415 35428 20 0.0 35414.6 3.7X -count() 11114 11128 16 0.1 11113.6 11.7X -Select 100 columns, one bad input field 93353 93670 275 0.0 93352.6 1.4X -Select 100 columns, corrupt record field 113569 113952 373 0.0 113568.8 1.1X +Select 1000 columns 129509 130323 1388 0.0 129509.4 1.0X +Select 100 columns 42474 42572 108 0.0 42473.6 3.0X +Select one column 35479 35586 93 0.0 35479.1 3.7X +count() 11021 11071 47 0.1 11021.3 11.8X +Select 100 columns, one bad input field 94652 94795 134 0.0 94652.0 1.4X +Select 100 columns, corrupt record field 115336 115542 350 0.0 115336.0 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 18498 18589 87 0.5 1849.8 1.0X -Select 1 column + count() 11078 11095 27 0.9 1107.8 1.7X -count() 3928 3950 22 2.5 392.8 4.7X +Select 10 columns + count() 19959 20022 76 0.5 1995.9 1.0X +Select 1 column + count() 13920 13968 54 0.7 1392.0 1.4X +count() 3928 3938 11 2.5 392.8 5.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 1933 1940 11 5.2 193.3 1.0X -to_csv(timestamp) 18078 18243 255 0.6 1807.8 0.1X -write timestamps to files 12668 12786 134 0.8 1266.8 0.2X -Create a dataset of dates 2196 2201 5 4.6 219.6 0.9X -to_csv(date) 9583 9597 21 1.0 958.3 0.2X -write dates to files 7091 7110 20 1.4 709.1 0.3X +Create a dataset of timestamps 1940 1977 56 5.2 194.0 1.0X +to_csv(timestamp) 15398 15669 458 0.6 1539.8 0.1X +write timestamps to files 12438 12454 19 0.8 1243.8 0.2X +Create a dataset of dates 2157 2171 18 4.6 215.7 0.9X +to_csv(date) 11764 11839 95 0.9 1176.4 0.2X +write dates to files 8893 8907 12 1.1 889.3 0.2X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2166 2177 10 4.6 216.6 1.0X -read timestamps from files 53212 53402 281 0.2 5321.2 0.0X -infer timestamps from files 109788 110372 570 0.1 10978.8 0.0X -read date text from files 1921 1929 8 5.2 192.1 1.1X -read date from files 25470 25499 25 0.4 2547.0 0.1X -infer date from files 27201 27342 134 0.4 2720.1 0.1X -timestamp strings 3638 3653 19 2.7 363.8 0.6X -parse timestamps from Dataset[String] 61894 62532 555 0.2 6189.4 0.0X -infer timestamps from Dataset[String] 125171 125430 236 0.1 12517.1 0.0X -date strings 3736 3749 14 2.7 373.6 0.6X -parse dates from Dataset[String] 30787 30829 43 0.3 3078.7 0.1X -from_csv(timestamp) 60842 61035 209 0.2 6084.2 0.0X -from_csv(date) 30123 30196 95 0.3 3012.3 0.1X +read timestamp text from files 2219 2230 11 4.5 221.9 1.0X +read timestamps from files 51519 51725 192 0.2 5151.9 0.0X +infer timestamps from files 104744 104885 124 0.1 10474.4 0.0X +read date text from files 1940 1943 4 5.2 194.0 1.1X +read date from files 27099 27118 33 0.4 2709.9 0.1X +infer date from files 27662 27703 61 0.4 2766.2 0.1X +timestamp strings 4225 4242 15 2.4 422.5 0.5X +parse timestamps from Dataset[String] 56090 56479 376 0.2 5609.0 0.0X +infer timestamps from Dataset[String] 115629 116245 1049 0.1 11562.9 0.0X +date strings 4337 4344 10 2.3 433.7 0.5X +parse dates from Dataset[String] 32373 32476 120 0.3 3237.3 0.1X +from_csv(timestamp) 54952 55157 300 0.2 5495.2 0.0X +from_csv(date) 30924 30985 66 0.3 3092.4 0.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 28985 29042 80 0.0 289852.9 1.0X -pushdown disabled 29080 29146 58 0.0 290799.4 1.0X -w/ filters 2072 2084 17 0.0 20722.3 14.0X +w/o filters 25630 25636 8 0.0 256301.4 1.0X +pushdown disabled 25673 25681 9 0.0 256734.0 1.0X +w/ filters 1873 1886 15 0.1 18733.1 13.7X diff --git a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt index 2d506f03d9f7e..6f68d60ce619a 100644 --- a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt @@ -7,106 +7,114 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 73307 73400 141 1.4 733.1 1.0X -UTF-8 is set 143834 143925 152 0.7 1438.3 0.5X +No encoding 70753 71127 471 1.4 707.5 1.0X +UTF-8 is set 128105 129183 1165 0.8 1281.1 0.6X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 50894 51065 292 2.0 508.9 1.0X -UTF-8 is set 98462 99455 1173 1.0 984.6 0.5X +No encoding 59588 59643 73 1.7 595.9 1.0X +UTF-8 is set 97081 97122 62 1.0 970.8 0.6X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 64011 64969 1001 0.2 6401.1 1.0X -UTF-8 is set 102757 102984 311 0.1 10275.7 0.6X +No encoding 58835 59259 659 0.2 5883.5 1.0X +UTF-8 is set 103117 103218 88 0.1 10311.7 0.6X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 132559 133561 1010 0.0 265117.3 1.0X -UTF-8 is set 151458 152129 611 0.0 302915.4 0.9X +No encoding 142993 143485 436 0.0 285985.3 1.0X +UTF-8 is set 165446 165496 60 0.0 330892.4 0.9X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 21148 21202 87 0.5 2114.8 1.0X -Select 1 column 24701 24724 21 0.4 2470.1 0.9X +Select 10 columns 21557 21593 61 0.5 2155.7 1.0X +Select 1 column 24197 24236 35 0.4 2419.7 0.9X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 6945 6998 59 1.4 694.5 1.0X -Short column with UTF-8 11510 11569 51 0.9 1151.0 0.6X -Wide column without encoding 95004 95795 790 0.1 9500.4 0.1X -Wide column with UTF-8 149223 149409 276 0.1 14922.3 0.0X +Short column without encoding 9795 9820 29 1.0 979.5 1.0X +Short column with UTF-8 16442 16536 146 0.6 1644.2 0.6X +Wide column without encoding 99134 99475 300 0.1 9913.4 0.1X +Wide column with UTF-8 155913 156369 692 0.1 15591.3 0.1X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 649 652 3 15.4 64.9 1.0X -from_json 22284 22393 99 0.4 2228.4 0.0X -json_tuple 32310 32824 484 0.3 3231.0 0.0X -get_json_object 22111 22751 568 0.5 2211.1 0.0X +Text read 671 679 7 14.9 67.1 1.0X +from_json 25356 25432 79 0.4 2535.6 0.0X +json_tuple 29464 29927 672 0.3 2946.4 0.0X +get_json_object 21841 21877 32 0.5 2184.1 0.0X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 2894 2903 8 17.3 57.9 1.0X -schema inferring 26724 26785 62 1.9 534.5 0.1X -parsing 37502 37632 131 1.3 750.0 0.1X +Text read 3109 3116 12 16.1 62.2 1.0X +schema inferring 28751 28765 15 1.7 575.0 0.1X +parsing 34923 35030 151 1.4 698.5 0.1X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 10994 11010 16 4.5 219.9 1.0X -Schema inferring 45654 45677 37 1.1 913.1 0.2X -Parsing without charset 34476 34559 73 1.5 689.5 0.3X -Parsing with UTF-8 56987 57002 13 0.9 1139.7 0.2X +Text read 10787 10818 32 4.6 215.7 1.0X +Schema inferring 49577 49775 184 1.0 991.5 0.2X +Parsing without charset 35343 35433 87 1.4 706.9 0.3X +Parsing with UTF-8 60253 60290 35 0.8 1205.1 0.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2150 2188 35 4.7 215.0 1.0X -to_json(timestamp) 17874 18080 294 0.6 1787.4 0.1X -write timestamps to files 12518 12538 34 0.8 1251.8 0.2X -Create a dataset of dates 2298 2310 18 4.4 229.8 0.9X -to_json(date) 11673 11703 27 0.9 1167.3 0.2X -write dates to files 7121 7135 12 1.4 712.1 0.3X +Create a dataset of timestamps 2200 2209 8 4.5 220.0 1.0X +to_json(timestamp) 18410 18602 264 0.5 1841.0 0.1X +write timestamps to files 11841 12032 305 0.8 1184.1 0.2X +Create a dataset of dates 2353 2363 9 4.3 235.3 0.9X +to_json(date) 12135 12182 72 0.8 1213.5 0.2X +write dates to files 6776 6801 33 1.5 677.6 0.3X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2616 2641 34 3.8 261.6 1.0X -read timestamps from files 37481 37517 58 0.3 3748.1 0.1X -infer timestamps from files 84774 84964 201 0.1 8477.4 0.0X -read date text from files 2362 2365 3 4.2 236.2 1.1X -read date from files 16583 16612 29 0.6 1658.3 0.2X -timestamp strings 3927 3963 40 2.5 392.7 0.7X -parse timestamps from Dataset[String] 52827 53004 243 0.2 5282.7 0.0X -infer timestamps from Dataset[String] 101108 101644 769 0.1 10110.8 0.0X -date strings 4886 4906 26 2.0 488.6 0.5X -parse dates from Dataset[String] 27623 27694 62 0.4 2762.3 0.1X -from_json(timestamp) 71764 71887 124 0.1 7176.4 0.0X -from_json(date) 46200 46314 99 0.2 4620.0 0.1X +read timestamp text from files 2563 2580 20 3.9 256.3 1.0X +read timestamps from files 41261 41360 97 0.2 4126.1 0.1X +infer timestamps from files 92292 92517 243 0.1 9229.2 0.0X +read date text from files 2332 2340 11 4.3 233.2 1.1X +read date from files 18753 18768 13 0.5 1875.3 0.1X +timestamp strings 3108 3123 13 3.2 310.8 0.8X +parse timestamps from Dataset[String] 51078 51448 323 0.2 5107.8 0.1X +infer timestamps from Dataset[String] 101373 101429 65 0.1 10137.3 0.0X +date strings 4126 4138 15 2.4 412.6 0.6X +parse dates from Dataset[String] 29365 29398 30 0.3 2936.5 0.1X +from_json(timestamp) 67033 67098 63 0.1 6703.3 0.0X +from_json(date) 44495 44581 125 0.2 4449.5 0.1X + +OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 30167 30223 48 0.0 301674.9 1.0X +pushdown disabled 30291 30311 30 0.0 302914.8 1.0X +w/ filters 901 915 14 0.1 9012.4 33.5X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index c22118f91b3fc..38ad9d0077f9a 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -7,106 +7,114 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 63839 64000 263 1.6 638.4 1.0X -UTF-8 is set 124633 124945 429 0.8 1246.3 0.5X +No encoding 78058 78116 76 1.3 780.6 1.0X +UTF-8 is set 125709 126521 1367 0.8 1257.1 0.6X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 51720 51901 157 1.9 517.2 1.0X -UTF-8 is set 91161 91190 25 1.1 911.6 0.6X +No encoding 60424 60567 188 1.7 604.2 1.0X +UTF-8 is set 92714 92864 140 1.1 927.1 0.7X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 58486 59038 714 0.2 5848.6 1.0X -UTF-8 is set 103045 103350 358 0.1 10304.5 0.6X +No encoding 65047 65761 662 0.2 6504.7 1.0X +UTF-8 is set 101823 101918 113 0.1 10182.3 0.6X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 134909 135024 105 0.0 269818.6 1.0X -UTF-8 is set 154418 154593 155 0.0 308836.7 0.9X +No encoding 145471 146067 601 0.0 290941.4 1.0X +UTF-8 is set 158504 159237 635 0.0 317008.4 0.9X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 19538 19620 70 0.5 1953.8 1.0X -Select 1 column 26142 26159 15 0.4 2614.2 0.7X +Select 10 columns 21386 21451 112 0.5 2138.6 1.0X +Select 1 column 27172 27214 58 0.4 2717.2 0.8X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 8103 8162 53 1.2 810.3 1.0X -Short column with UTF-8 13104 13150 58 0.8 1310.4 0.6X -Wide column without encoding 135280 135593 375 0.1 13528.0 0.1X -Wide column with UTF-8 175189 175483 278 0.1 17518.9 0.0X +Short column without encoding 9283 9363 69 1.1 928.3 1.0X +Short column with UTF-8 15330 15369 61 0.7 1533.0 0.6X +Wide column without encoding 138885 139153 239 0.1 13888.5 0.1X +Wide column with UTF-8 177201 177650 501 0.1 17720.1 0.1X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 1225 1234 8 8.2 122.5 1.0X -from_json 22482 22552 95 0.4 2248.2 0.1X -json_tuple 30203 30338 146 0.3 3020.3 0.0X -get_json_object 22219 22245 26 0.5 2221.9 0.1X +Text read 1224 1243 17 8.2 122.4 1.0X +from_json 25191 25327 214 0.4 2519.1 0.0X +json_tuple 30333 30380 42 0.3 3033.3 0.0X +get_json_object 21611 21739 112 0.5 2161.1 0.1X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 5897 5904 10 8.5 117.9 1.0X -schema inferring 30282 30340 50 1.7 605.6 0.2X -parsing 33304 33577 289 1.5 666.1 0.2X +Text read 5923 5941 32 8.4 118.5 1.0X +schema inferring 34089 34238 135 1.5 681.8 0.2X +parsing 44699 45952 1108 1.1 894.0 0.1X Preparing data for benchmarking ... OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 9710 9757 80 5.1 194.2 1.0X -Schema inferring 35929 35939 9 1.4 718.6 0.3X -Parsing without charset 39175 39227 87 1.3 783.5 0.2X -Parsing with UTF-8 59188 59294 109 0.8 1183.8 0.2X +Text read 9727 9776 50 5.1 194.5 1.0X +Schema inferring 52529 52643 98 1.0 1050.6 0.2X +Parsing without charset 44563 44692 132 1.1 891.3 0.2X +Parsing with UTF-8 55558 55755 218 0.9 1111.2 0.2X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 1967 1977 9 5.1 196.7 1.0X -to_json(timestamp) 17086 17304 371 0.6 1708.6 0.1X -write timestamps to files 12691 12716 28 0.8 1269.1 0.2X -Create a dataset of dates 2192 2217 39 4.6 219.2 0.9X -to_json(date) 10541 10656 137 0.9 1054.1 0.2X -write dates to files 7259 7311 46 1.4 725.9 0.3X +Create a dataset of timestamps 1945 1964 22 5.1 194.5 1.0X +to_json(timestamp) 17990 18135 249 0.6 1799.0 0.1X +write timestamps to files 13198 13234 45 0.8 1319.8 0.1X +Create a dataset of dates 2202 2213 11 4.5 220.2 0.9X +to_json(date) 11219 11240 29 0.9 1121.9 0.2X +write dates to files 6932 6966 32 1.4 693.2 0.3X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2318 2326 13 4.3 231.8 1.0X -read timestamps from files 43345 43627 258 0.2 4334.5 0.1X -infer timestamps from files 89570 89621 59 0.1 8957.0 0.0X -read date text from files 2099 2107 9 4.8 209.9 1.1X -read date from files 18000 18065 98 0.6 1800.0 0.1X -timestamp strings 3937 3956 32 2.5 393.7 0.6X -parse timestamps from Dataset[String] 56001 56429 539 0.2 5600.1 0.0X -infer timestamps from Dataset[String] 109410 109963 559 0.1 10941.0 0.0X -date strings 4530 4540 9 2.2 453.0 0.5X -parse dates from Dataset[String] 29723 29767 72 0.3 2972.3 0.1X -from_json(timestamp) 74106 74619 728 0.1 7410.6 0.0X -from_json(date) 46599 46632 32 0.2 4659.9 0.0X +read timestamp text from files 2354 2368 12 4.2 235.4 1.0X +read timestamps from files 43681 43771 100 0.2 4368.1 0.1X +infer timestamps from files 90608 90771 161 0.1 9060.8 0.0X +read date text from files 2121 2129 9 4.7 212.1 1.1X +read date from files 19069 19103 32 0.5 1906.9 0.1X +timestamp strings 3943 3967 24 2.5 394.3 0.6X +parse timestamps from Dataset[String] 55239 55324 74 0.2 5523.9 0.0X +infer timestamps from Dataset[String] 106155 106258 99 0.1 10615.5 0.0X +date strings 4567 4572 5 2.2 456.7 0.5X +parse dates from Dataset[String] 31258 31461 321 0.3 3125.8 0.1X +from_json(timestamp) 76499 77031 504 0.1 7649.9 0.0X +from_json(date) 44188 44199 9 0.2 4418.8 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 30314 30334 28 0.0 303139.1 1.0X +pushdown disabled 30394 30429 54 0.0 303944.7 1.0X +w/ filters 906 913 8 0.1 9059.1 33.5X 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 a7b3d08ec4df7..38031002cfad3 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 @@ -31,7 +31,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} -import org.apache.spark.sql.catalyst.util.FailureSafeParser +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, FailureSafeParser} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsCatalogOptions, SupportsRead} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils @@ -94,6 +94,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
      *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -121,6 +124,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Boolean): DataFrameReader = option(key, value.toString) @@ -128,6 +134,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Long): DataFrameReader = option(key, value.toString) @@ -135,6 +144,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds an input option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Double): DataFrameReader = option(key, value.toString) @@ -142,6 +154,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * (Scala-specific) Adds input options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
        *
      • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -169,6 +184,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Adds input options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
          *
        • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -238,7 +256,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { Some("paths" -> objectMapper.writeValueAsString(paths.toArray)) } - val finalOptions = sessionOptions ++ extraOptions.toMap ++ pathsOption + val finalOptions = sessionOptions ++ extraOptions.originalMap ++ pathsOption val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) val (table, catalog, ident) = provider match { case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty => @@ -276,7 +294,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { paths = paths, userSpecifiedSchema = userSpecifiedSchema, className = source, - options = extraOptions.toMap).resolveRelation()) + options = extraOptions.originalMap).resolveRelation()) } /** @@ -290,7 +308,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { // properties should override settings in extraOptions. this.extraOptions ++= properties.asScala // explicit url and dbtable should override all - this.extraOptions += (JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) + this.extraOptions ++= Seq(JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) format("jdbc").load() } @@ -361,7 +379,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { connectionProperties: Properties): DataFrame = { assertNoSpecifiedSchema("jdbc") // connectionProperties should override settings in extraOptions. - val params = extraOptions.toMap ++ connectionProperties.asScala.toMap + val params = extraOptions ++ connectionProperties.asScala val options = new JDBCOptions(url, table, params) val parts: Array[Partition] = predicates.zipWithIndex.map { case (part, i) => JDBCPartition(part, i) : Partition @@ -451,6 +469,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * It does not change the behavior of partition discovery.
        • *
        • `recursiveFileLookup`: recursively scan a directory for files. Using this option * disables partition discovery
        • + *
        • `allowNonNumericNumbers` (default `true`): allows JSON parser to recognize set of + * "Not-a-Number" (NaN) tokens as legal floating number values: + *
            + *
          • `+INF` for positive infinity, as well as alias of `+Infinity` and `Infinity`. + *
          • `-INF` for negative infinity), alias `-Infinity`. + *
          • `NaN` for other not-a-numbers, like result of division by zero. + *
          + *
        • *
        * * @since 2.0.0 @@ -879,6 +905,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { private var userSpecifiedSchema: Option[StructType] = None - private val extraOptions = new scala.collection.mutable.HashMap[String, String] + private var extraOptions = CaseInsensitiveMap[String](Map.empty) } 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 9aef382470bcc..f463166a9f268 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchT import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateTableAsSelectStatement, InsertIntoStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelectStatement} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.{CatalogPlugin, CatalogV2Implicits, CatalogV2Util, Identifier, SupportsCatalogOptions, Table, TableCatalog, TableProvider, V1Table} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} @@ -104,6 +105,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
          *
        • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -131,6 +135,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Boolean): DataFrameWriter[T] = option(key, value.toString) @@ -138,6 +145,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Long): DataFrameWriter[T] = option(key, value.toString) @@ -145,6 +155,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds an output option for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * @since 2.0.0 */ def option(key: String, value: Double): DataFrameWriter[T] = option(key, value.toString) @@ -152,6 +165,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * (Scala-specific) Adds output options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
            *
          • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -179,6 +195,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Adds output options for the underlying data source. * + * All options are maintained in a case-insensitive way in terms of key names. + * If a new option has the same key case-insensitively, it will override the existing option. + * * You can set the following option(s): *
              *
            • `timeZone` (default session local timezone): sets the string that indicates a time zone ID @@ -287,7 +306,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val provider = maybeV2Provider.get val sessionOptions = DataSourceV2Utils.extractSessionConfigs( provider, df.sparkSession.sessionState.conf) - val options = sessionOptions ++ extraOptions + val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap val dsOptions = new CaseInsensitiveStringMap(options.asJava) def getTable: Table = { @@ -768,7 +787,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { // connectionProperties should override settings in extraOptions. this.extraOptions ++= connectionProperties.asScala // explicit url and dbtable should override all - this.extraOptions += ("url" -> url, "dbtable" -> table) + this.extraOptions ++= Seq("url" -> url, "dbtable" -> table) format("jdbc").save() } @@ -960,7 +979,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private var mode: SaveMode = SaveMode.ErrorIfExists - private val extraOptions = new scala.collection.mutable.HashMap[String, String] + private var extraOptions = CaseInsensitiveMap[String](Map.empty) private var partitioningColumns: Option[Seq[String]] = None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index d5501326397c9..7b0bae6a82053 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -227,7 +227,7 @@ class Dataset[T] private[sql]( val plan = queryExecution.analyzed match { case c: Command => LocalRelation(c.output, withAction("command", queryExecution)(_.executeCollect())) - case u @ Union(children) if children.forall(_.isInstanceOf[Command]) => + case u @ Union(children, _, _) if children.forall(_.isInstanceOf[Command]) => LocalRelation(u.output, withAction("command", queryExecution)(_.executeCollect())) case _ => queryExecution.analyzed @@ -2071,51 +2071,9 @@ class Dataset[T] private[sql]( * @since 3.1.0 */ def unionByName(other: Dataset[T], allowMissingColumns: Boolean): Dataset[T] = withSetOperator { - // Check column name duplication - val resolver = sparkSession.sessionState.analyzer.resolver - val leftOutputAttrs = logicalPlan.output - val rightOutputAttrs = other.logicalPlan.output - - SchemaUtils.checkColumnNameDuplication( - leftOutputAttrs.map(_.name), - "in the left attributes", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - SchemaUtils.checkColumnNameDuplication( - rightOutputAttrs.map(_.name), - "in the right attributes", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - - // Builds a project list for `other` based on `logicalPlan` output names - val rightProjectList = leftOutputAttrs.map { lattr => - rightOutputAttrs.find { rattr => resolver(lattr.name, rattr.name) }.getOrElse { - if (allowMissingColumns) { - Alias(Literal(null, lattr.dataType), lattr.name)() - } else { - throw new AnalysisException( - s"""Cannot resolve column name "${lattr.name}" among """ + - s"""(${rightOutputAttrs.map(_.name).mkString(", ")})""") - } - } - } - - // Delegates failure checks to `CheckAnalysis` - val notFoundAttrs = rightOutputAttrs.diff(rightProjectList) - val rightChild = Project(rightProjectList ++ notFoundAttrs, other.logicalPlan) - - // Builds a project for `logicalPlan` based on `other` output names, if allowing - // missing columns. - val leftChild = if (allowMissingColumns) { - val missingAttrs = notFoundAttrs.map { attr => - Alias(Literal(null, attr.dataType), attr.name)() - } - Project(leftOutputAttrs ++ missingAttrs, logicalPlan) - } else { - logicalPlan - } - // This breaks caching, but it's usually ok because it addresses a very specific use case: // using union to union many files or partitions. - CombineUnions(Union(leftChild, rightChild)) + CombineUnions(Union(logicalPlan :: other.logicalPlan :: Nil, true, allowMissingColumns)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 08b0a1c6a60a2..e5d53f5fd4c65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -29,6 +29,7 @@ import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog @@ -900,7 +901,13 @@ object SparkSession extends Logging { * @since 2.0.0 */ def getOrCreate(): SparkSession = synchronized { - assertOnDriver() + val sparkConf = new SparkConf() + options.foreach { case (k, v) => sparkConf.set(k, v) } + + if (!sparkConf.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { + assertOnDriver() + } + // Get the session from current thread's active session. var session = activeThreadSession.get() if ((session ne null) && !session.sparkContext.isStopped) { @@ -919,9 +926,6 @@ object SparkSession extends Logging { // No active nor global default session. Create a new one. val sparkContext = userSuppliedContext.getOrElse { - val sparkConf = new SparkConf() - options.foreach { case (k, v) => sparkConf.set(k, v) } - // set a random app name if not given. if (!sparkConf.contains("spark.app.name")) { sparkConf.setAppName(java.util.UUID.randomUUID().toString) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index ff706b5061f0a..6952f4bfd0566 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.ColumnarRule +import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} /** * :: Experimental :: @@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.ColumnarRule *
            • Customized Parser.
            • *
            • (External) Catalog listeners.
            • *
            • Columnar Rules.
            • + *
            • Adaptive Query Stage Preparation Rules.
            • *
            * * The extensions can be used by calling `withExtensions` on the [[SparkSession.Builder]], for @@ -96,8 +97,10 @@ class SparkSessionExtensions { type ParserBuilder = (SparkSession, ParserInterface) => ParserInterface type FunctionDescription = (FunctionIdentifier, ExpressionInfo, FunctionBuilder) type ColumnarRuleBuilder = SparkSession => ColumnarRule + type QueryStagePrepRuleBuilder = SparkSession => Rule[SparkPlan] private[this] val columnarRuleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder] + private[this] val queryStagePrepRuleBuilders = mutable.Buffer.empty[QueryStagePrepRuleBuilder] /** * Build the override rules for columnar execution. @@ -106,6 +109,13 @@ class SparkSessionExtensions { columnarRuleBuilders.map(_.apply(session)).toSeq } + /** + * Build the override rules for the query stage preparation phase of adaptive query execution. + */ + private[sql] def buildQueryStagePrepRules(session: SparkSession): Seq[Rule[SparkPlan]] = { + queryStagePrepRuleBuilders.map(_.apply(session)).toSeq + } + /** * Inject a rule that can override the columnar execution of an executor. */ @@ -113,6 +123,14 @@ class SparkSessionExtensions { columnarRuleBuilders += builder } + /** + * Inject a rule that can override the the query stage preparation phase of adaptive query + * execution. + */ + def injectQueryStagePrepRule(builder: QueryStagePrepRuleBuilder): Unit = { + queryStagePrepRuleBuilders += builder + } + private[this] val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index bc3f38a35834d..8923d5c86e19a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -577,25 +577,20 @@ class ResolveSessionCatalog( case ShowTableProperties(r: ResolvedView, propertyKey) => ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey) - case DescribeFunctionStatement(nameParts, extended) => - val functionIdent = - parseSessionCatalogFunctionIdentifier(nameParts, "DESCRIBE FUNCTION") - DescribeFunctionCommand(functionIdent, extended) - - case ShowFunctionsStatement(userScope, systemScope, pattern, fun) => - val (database, function) = fun match { - case Some(nameParts) => - val FunctionIdentifier(fn, db) = - parseSessionCatalogFunctionIdentifier(nameParts, "SHOW FUNCTIONS") - (db, Some(fn)) - case None => (None, pattern) - } - ShowFunctionsCommand(database, function, userScope, systemScope) + case DescribeFunction(ResolvedFunc(identifier), extended) => + DescribeFunctionCommand(identifier.asFunctionIdentifier, extended) + + case ShowFunctions(None, userScope, systemScope, pattern) => + ShowFunctionsCommand(None, pattern, userScope, systemScope) + + case ShowFunctions(Some(ResolvedFunc(identifier)), userScope, systemScope, _) => + val funcIdentifier = identifier.asFunctionIdentifier + ShowFunctionsCommand( + funcIdentifier.database, Some(funcIdentifier.funcName), userScope, systemScope) - case DropFunctionStatement(nameParts, ifExists, isTemp) => - val FunctionIdentifier(function, database) = - parseSessionCatalogFunctionIdentifier(nameParts, "DROP FUNCTION") - DropFunctionCommand(database, function, ifExists, isTemp) + case DropFunction(ResolvedFunc(identifier), ifExists, isTemp) => + val funcIdentifier = identifier.asFunctionIdentifier + DropFunctionCommand(funcIdentifier.database, funcIdentifier.funcName, ifExists, isTemp) case CreateFunctionStatement(nameParts, className, resources, isTemp, ignoreIfExists, replace) => @@ -618,37 +613,15 @@ class ResolveSessionCatalog( replace) } else { val FunctionIdentifier(function, database) = - parseSessionCatalogFunctionIdentifier(nameParts, "CREATE FUNCTION") + parseSessionCatalogFunctionIdentifier(nameParts) CreateFunctionCommand(database, function, className, resources, isTemp, ignoreIfExists, replace) } - } - - // TODO: move function related v2 statements to the new framework. - private def parseSessionCatalogFunctionIdentifier( - nameParts: Seq[String], - sql: String): FunctionIdentifier = { - if (nameParts.length == 1 && isTempFunction(nameParts.head)) { - return FunctionIdentifier(nameParts.head) - } - nameParts match { - case SessionCatalogAndIdentifier(_, ident) => - if (nameParts.length == 1) { - // If there is only one name part, it means the current catalog is the session catalog. - // Here we don't fill the default database, to keep the error message unchanged for - // v1 commands. - FunctionIdentifier(nameParts.head, None) - } else { - ident.namespace match { - case Array(db) => FunctionIdentifier(ident.name, Some(db)) - case _ => - throw new AnalysisException(s"Unsupported function name '$ident'") - } - } - - case _ => throw new AnalysisException(s"$sql is only supported in v1 catalog") - } + case RefreshFunction(ResolvedFunc(identifier)) => + // Fallback to v1 command + val funcIdentifier = identifier.asFunctionIdentifier + RefreshFunctionCommand(funcIdentifier.database, funcIdentifier.funcName) } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = tableName match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala new file mode 100644 index 0000000000000..1113e63cab332 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -0,0 +1,343 @@ +/* + * 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.columnar + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, LessThanOrEqual, Literal, Or, Predicate, StartsWith} +import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.StorageLevel + +/** + * Basic interface that all cached batches of data must support. This is primarily to allow + * for metrics to be handled outside of the encoding and decoding steps in a standard way. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatch { + def numRows: Int + def sizeInBytes: Long +} + +/** + * Provides APIs that handle transformations of SQL data associated with the cache/persist APIs. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatchSerializer extends Serializable { + /** + * Can `convertColumnarBatchToCachedBatch()` be called instead of + * `convertInternalRowToCachedBatch()` for this given schema? True if it can and false if it + * cannot. Columnar input is only supported if the plan could produce columnar output. Currently + * this is mostly supported by input formats like parquet and orc, but more operations are likely + * to be supported soon. + * @param schema the schema of the data being stored. + * @return True if columnar input can be supported, else false. + */ + def supportsColumnarInput(schema: Seq[Attribute]): Boolean + + /** + * Convert an `RDD[InternalRow]` into an `RDD[CachedBatch]` in preparation for caching the data. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Convert an `RDD[ColumnarBatch]` into an `RDD[CachedBatch]` in preparation for caching the data. + * This will only be called if `supportsColumnarInput()` returned true for the given schema and + * the plan up to this point would could produce columnar output without modifying it. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Builds a function that can be used to filter batches prior to being decompressed. + * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide the filter logic + * necessary. You will need to provide metrics for this to work. [[SimpleMetricsCachedBatch]] + * provides the APIs to hold those metrics and explains the metrics used, really just min and max. + * Note that this is intended to skip batches that are not needed, and the actual filtering of + * individual rows is handled later. + * @param predicates the set of expressions to use for filtering. + * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful + * if you don't store it with the data. + * @return a function that takes the partition id and the iterator of batches in the partition. + * It returns an iterator of batches that should be decompressed. + */ + def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] + + /** + * Can `convertCachedBatchToColumnarBatch()` be called instead of + * `convertCachedBatchToInternalRow()` for this given schema? True if it can and false if it + * cannot. Columnar output is typically preferred because it is more efficient. Note that + * `convertCachedBatchToInternalRow()` must always be supported as there are other checks that + * can force row based output. + * @param schema the schema of the data being checked. + * @return true if columnar output should be used for this schema, else false. + */ + def supportsColumnarOutput(schema: StructType): Boolean + + /** + * The exact java types of the columns that are output in columnar processing mode. This + * is a performance optimization for code generation and is optional. + * @param attributes the attributes to be output. + * @param conf the config for the query that will read the data. + */ + def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = None + + /** + * Convert the cached data into a ColumnarBatch. This currently is only used if + * `supportsColumnarOutput()` returns true for the associated schema, but there are other checks + * that can force row based output. One of the main advantages of doing columnar output over row + * based output is that the code generation is more standard and can be combined with code + * generation for downstream operations. + * @param input the cached batches that should be converted. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the fields that should be loaded from the data and the order they + * should appear in the output batch. + * @param conf the configuration for the job. + * @return an RDD of the input cached batches transformed into the ColumnarBatch format. + */ + def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] + + /** + * Convert the cached batch into `InternalRow`s. If you want this to be performant, code + * generation is advised. + * @param input the cached batches that should be converted. + * @param cacheAttributes the attributes of the data in the batch. + * @param selectedAttributes the field that should be loaded from the data and the order they + * should appear in the output rows. + * @param conf the configuration for the job. + * @return RDD of the rows that were stored in the cached batches. + */ + def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] +} + +/** + * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with + * the [[SimpleMetricsCachedBatchSerializer]]. + * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata + * are needed in the row. + */ +@DeveloperApi +@Since("3.1.0") +trait SimpleMetricsCachedBatch extends CachedBatch { + /** + * Holds stats for each cached column. The optional `upperBound` and `lowerBound` should be + * of the same type as the original column. If they are null, then it is assumed that they + * are not provided, and will not be used for filtering. + *
              + *
            • `upperBound` (optional)
            • + *
            • `lowerBound` (Optional)
            • + *
            • `nullCount`: `Int`
            • + *
            • `rowCount`: `Int`
            • + *
            • `sizeInBytes`: `Long`
            • + *
            + * These are repeated for each column in the original cached data. + */ + val stats: InternalRow + override def sizeInBytes: Long = + Range.apply(4, stats.numFields, 5).map(stats.getLong).sum +} + +// Currently, uses statistics for all atomic types that are not `BinaryType`. +private object ExtractableLiteral { + def unapply(expr: Expression): Option[Literal] = expr match { + case lit: Literal => lit.dataType match { + case BinaryType => None + case _: AtomicType => Some(lit) + case _ => None + } + case _ => None + } +} + +/** + * Provides basic filtering for [[CachedBatchSerializer]] implementations. + * The requirement to extend this is that all of the batches produced by your serializer are + * instances of [[SimpleMetricsCachedBatch]]. + * This does not calculate the metrics needed to be stored in the batches. That is up to each + * implementation. The metrics required are really just min and max values and those are optional + * especially for complex types. Because those metrics are simple and it is likely that compression + * will also be done on the data we thought it best to let each implementation decide on the most + * efficient way to calculate the metrics, possibly combining them with compression passes that + * might also be done across the data. + */ +@DeveloperApi +@Since("3.1.0") +abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging { + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + val stats = new PartitionStatistics(cachedAttributes) + val statsSchema = stats.schema + + def statsFor(a: Attribute): ColumnStatisticsSchema = { + stats.forAttribute(a) + } + + // Returned filter predicate should return false iff it is impossible for the input expression + // to evaluate to `true` based on statistics collected about this partition batch. + @transient lazy val buildFilter: PartialFunction[Expression, Expression] = { + case And(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => + (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) + + case Or(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) || buildFilter(rhs) + + case EqualTo(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualTo(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound + + case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l + case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound + + case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound <= l + case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + l <= statsFor(a).upperBound + + case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound + case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => + l <= statsFor(a).upperBound + case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => + statsFor(a).lowerBound <= l + + case IsNull(a: Attribute) => statsFor(a).nullCount > 0 + case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 + + case In(a: AttributeReference, list: Seq[Expression]) + if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => + list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && + l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) + // This is an example to explain how it works, imagine that the id column stored as follows: + // __________________________________________ + // | Partition ID | lowerBound | upperBound | + // |--------------|------------|------------| + // | p1 | '1' | '9' | + // | p2 | '10' | '19' | + // | p3 | '20' | '29' | + // | p4 | '30' | '39' | + // | p5 | '40' | '49' | + // |______________|____________|____________| + // + // A filter: df.filter($"id".startsWith("2")). + // In this case it substr lowerBound and upperBound: + // ________________________________________________________________________________________ + // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | + // |--------------|-----------------------------------|-----------------------------------| + // | p1 | '1' | '9' | + // | p2 | '1' | '1' | + // | p3 | '2' | '2' | + // | p4 | '3' | '3' | + // | p5 | '4' | '4' | + // |______________|___________________________________|___________________________________| + // + // We can see that we only need to read p1 and p3. + case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound.substr(0, Length(l)) <= l && + l <= statsFor(a).upperBound.substr(0, Length(l)) + } + + // When we bind the filters we need to do it against the stats schema + val partitionFilters: Seq[Expression] = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + statsSchema, + allowFailures = true)) + + boundFilter.foreach(_ => + filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) + + // If the filter can't be resolved then we are missing required statistics. + boundFilter.filter(_.resolved) + } + } + + def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = { + val partitionFilter = Predicate.create( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + cachedAttributes) + + partitionFilter.initialize(index) + val schemaIndex = cachedAttributes.zipWithIndex + + cachedBatchIterator.filter { cb => + val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch] + if (!partitionFilter.eval(cachedBatch.stats)) { + logDebug { + val statsString = schemaIndex.map { case (a, i) => + val value = cachedBatch.stats.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + s"Skipping partition based on stats $statsString" + } + false + } else { + true + } + } + } + ret + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 7d86c48015406..7201026b11b6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -27,11 +27,10 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.columnar.{DefaultCachedBatchSerializer, InMemoryRelation} import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -85,11 +84,9 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val inMemoryRelation = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(planToCache) InMemoryRelation( - sessionWithAqeOff.sessionState.conf.useCompression, - sessionWithAqeOff.sessionState.conf.columnBatchSize, storageLevel, - qe.executedPlan, - tableName, - optimizedPlan = qe.optimizedPlan) + storageLevel, + qe, + tableName) } this.synchronized { @@ -195,9 +192,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { val sessionWithAqeOff = getOrCloneSessionWithAqeOff(spark) val newCache = sessionWithAqeOff.withActive { val qe = sessionWithAqeOff.sessionState.executePlan(cd.plan) - InMemoryRelation( - cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = qe.executedPlan), - optimizedPlan = qe.optimizedPlan) + InMemoryRelation(cd.cachedRepresentation.cacheBuilder, qe) } val recomputedPlan = cd.copy(cachedRepresentation = newCache) this.synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index e01cd8598db0c..13ea609f7bfa6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -49,6 +49,13 @@ class ColumnarRule { def postColumnarTransitions: Rule[SparkPlan] = plan => plan } +/** + * A trait that is used as a tag to indicate a transition from columns to rows. This allows plugins + * to replace the current [[ColumnarToRowExec]] with an optimized version and still have operations + * that walk a spark plan looking for this type of transition properly match it. + */ +trait ColumnarToRowTransition extends UnaryExecNode + /** * Provides a common executor to translate an [[RDD]] of [[ColumnarBatch]] into an [[RDD]] of * [[InternalRow]]. This is inserted whenever such a transition is determined to be needed. @@ -57,7 +64,7 @@ class ColumnarRule { * [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]] and * [[MapPartitionsInRWithArrowExec]]. Eventually this should replace those implementations. */ -case class ColumnarToRowExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { +case class ColumnarToRowExec(child: SparkPlan) extends ColumnarToRowTransition with CodegenSupport { assert(child.supportsColumnar) override def output: Seq[Attribute] = child.output @@ -479,7 +486,9 @@ case class RowToColumnarExec(child: SparkPlan) extends UnaryExecNode { * Apply any user defined [[ColumnarRule]]s and find the correct place to insert transitions * to/from columnar formatted data. */ -case class ApplyColumnarRulesAndInsertTransitions(conf: SQLConf, columnarRules: Seq[ColumnarRule]) +case class ApplyColumnarRulesAndInsertTransitions( + conf: SQLConf, + columnarRules: Seq[ColumnarRule]) extends Rule[SparkPlan] { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 78808ff21394c..bef9f4b46c628 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -608,12 +608,20 @@ case class FileSourceScanExec( new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } + // Filters unused DynamicPruningExpression expressions - one which has been replaced + // with DynamicPruningExpression(Literal.TrueLiteral) during Physical Planning + private def filterUnusedDynamicPruningExpressions( + predicates: Seq[Expression]): Seq[Expression] = { + predicates.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral)) + } + override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, output.map(QueryPlan.normalizeExpressions(_, output)), requiredSchema, - QueryPlan.normalizePredicates(partitionFilters, output), + QueryPlan.normalizePredicates( + filterUnusedDynamicPruningExpressions(partitionFilters), output), optionalBucketSet, optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 791e432269632..e4b9322016cf0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInSortMergeJoin +import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -332,7 +332,7 @@ object QueryExecution { // as the original plan is hidden behind `AdaptiveSparkPlanExec`. adaptiveExecutionRule.toSeq ++ Seq( - CoalesceBucketsInSortMergeJoin(sparkSession.sessionState.conf), + CoalesceBucketsInJoin(sparkSession.sessionState.conf), PlanDynamicPruningFilters(sparkSession), PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 363282ea95997..129312160b1b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution -import java.util.Locale +import java.time.ZoneOffset +import java.util.{Locale, TimeZone} import javax.ws.rs.core.UriBuilder import scala.collection.JavaConverters._ @@ -32,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} @@ -56,6 +58,9 @@ class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser(conf) { class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { import org.apache.spark.sql.catalyst.parser.ParserUtils._ + private val configKeyValueDef = """([a-zA-Z_\d\\.:]+)\s*=(.*)""".r + private val configKeyDef = """([a-zA-Z_\d\\.:]+)$""".r + /** * Create a [[SetCommand]] logical plan. * @@ -64,17 +69,28 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * character in the raw string. */ override def visitSetConfiguration(ctx: SetConfigurationContext): LogicalPlan = withOrigin(ctx) { - // Construct the command. - val raw = remainder(ctx.SET.getSymbol) - val keyValueSeparatorIndex = raw.indexOf('=') - if (keyValueSeparatorIndex >= 0) { - val key = raw.substring(0, keyValueSeparatorIndex).trim - val value = raw.substring(keyValueSeparatorIndex + 1).trim - SetCommand(Some(key -> Option(value))) - } else if (raw.nonEmpty) { - SetCommand(Some(raw.trim -> None)) + remainder(ctx.SET.getSymbol).trim match { + case configKeyValueDef(key, value) => + SetCommand(Some(key -> Option(value.trim))) + case configKeyDef(key) => + SetCommand(Some(key -> None)) + case s if s == "-v" => + SetCommand(Some("-v" -> None)) + case s if s.isEmpty => + SetCommand(None) + case _ => throw new ParseException("Expected format is 'SET', 'SET key', or " + + "'SET key=value'. If you want to include special characters in key, " + + "please use quotes, e.g., SET `ke y`=value.", ctx) + } + } + + override def visitSetQuotedConfiguration(ctx: SetQuotedConfigurationContext) + : LogicalPlan = withOrigin(ctx) { + val keyStr = ctx.configKey().getText + if (ctx.EQ() != null) { + SetCommand(Some(keyStr -> Option(remainder(ctx.EQ().getSymbol).trim))) } else { - SetCommand(None) + SetCommand(Some(keyStr -> None)) } } @@ -83,11 +99,60 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Example SQL : * {{{ * RESET; + * RESET spark.sql.session.timeZone; * }}} */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { - ResetCommand + remainder(ctx.RESET.getSymbol).trim match { + case configKeyDef(key) => + ResetCommand(Some(key)) + case s if s.trim.isEmpty => + ResetCommand(None) + case _ => throw new ParseException("Expected format is 'RESET' or 'RESET key'. " + + "If you want to include special characters in key, " + + "please use quotes, e.g., RESET `ke y`.", ctx) + } + } + + override def visitResetQuotedConfiguration( + ctx: ResetQuotedConfigurationContext): LogicalPlan = withOrigin(ctx) { + ResetCommand(Some(ctx.configKey().getText)) + } + + /** + * Create a [[SetCommand]] logical plan to set [[SQLConf.SESSION_LOCAL_TIMEZONE]] + * Example SQL : + * {{{ + * SET TIME ZONE LOCAL; + * SET TIME ZONE 'Asia/Shanghai'; + * SET TIME ZONE INTERVAL 10 HOURS; + * }}} + */ + override def visitSetTimeZone(ctx: SetTimeZoneContext): LogicalPlan = withOrigin(ctx) { + val key = SQLConf.SESSION_LOCAL_TIMEZONE.key + if (ctx.interval != null) { + val interval = parseIntervalLiteral(ctx.interval) + if (interval.months != 0 || interval.days != 0 || + math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR || + interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) { + throw new ParseException("The interval value must be in the range of [-18, +18] hours" + + " with second precision", + ctx.interval()) + } else { + val seconds = (interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt + SetCommand(Some(key -> Some(ZoneOffset.ofTotalSeconds(seconds).toString))) + } + } else if (ctx.timezone != null) { + ctx.timezone.getType match { + case SqlBaseParser.LOCAL => + SetCommand(Some(key -> Some(TimeZone.getDefault.getID))) + case _ => + SetCommand(Some(key -> Some(string(ctx.STRING)))) + } + } else { + throw new ParseException("Invalid time zone displacement value", ctx) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7b5d8f15962d0..eb32bfcecae7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -232,6 +232,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { .orElse { if (hintToShuffleReplicateNL(hint)) createCartesianProduct() else None } .getOrElse(createJoinWithoutHint()) + case j @ ExtractSingleColumnNullAwareAntiJoin(leftKeys, rightKeys) => + Seq(joins.BroadcastHashJoinExec(leftKeys, rightKeys, LeftAnti, BuildRight, + None, planLater(j.left), planLater(j.right), isNullAwareAntiJoin = true)) + // If it is not an equi-join, we first look at the join hints w.r.t. the following order: // 1. broadcast hint: pick broadcast nested loop join. If both sides have the broadcast // hints, choose the smaller side (based on stats) to broadcast for inner and full joins, @@ -661,7 +665,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), - planLater(child), canChangeNumPartitions = false) :: Nil + planLater(child), noUserSpecifiedNumPartition = false) :: Nil } else { execution.CoalesceExec(numPartitions, planLater(child)) :: Nil } @@ -683,8 +687,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.LocalLimitExec(limit, planLater(child)) :: Nil case logical.GlobalLimit(IntegerLiteral(limit), child) => execution.GlobalLimitExec(limit, planLater(child)) :: Nil - case logical.Union(unionChildren) => - execution.UnionExec(unionChildren.map(planLater)) :: Nil + case union: logical.Union => + execution.UnionExec(union.children.map(planLater)) :: Nil case g @ logical.Generate(generator, _, outer, _, _, child) => execution.GenerateExec( generator, g.requiredChildOutput, outer, @@ -694,9 +698,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: logical.Range => execution.RangeExec(r) :: Nil case r: logical.RepartitionByExpression => - val canChangeNumParts = r.optNumPartitions.isEmpty exchange.ShuffleExchangeExec( - r.partitioning, planLater(r.child), canChangeNumParts) :: Nil + r.partitioning, + planLater(r.child), + noUserSpecifiedNumPartition = r.optNumPartitions.isEmpty) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil 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 558d990e8c4bb..59a42d8931925 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 @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -50,6 +50,7 @@ trait CodegenSupport extends SparkPlan { private def variablePrefix: String = this match { case _: HashAggregateExec => "agg" case _: BroadcastHashJoinExec => "bhj" + case _: ShuffledHashJoinExec => "shj" case _: SortMergeJoinExec => "smj" case _: RDDScanExec => "rdd" case _: DataSourceScanExec => "scan" @@ -903,6 +904,10 @@ case class CollapseCodegenStages( // The children of SortMergeJoin should do codegen separately. j.withNewChildren(j.children.map( child => InputAdapter(insertWholeStageCodegen(child)))) + case j: ShuffledHashJoinExec => + // The children of ShuffledHashJoin should do codegen separately. + j.withNewChildren(j.children.map( + child => InputAdapter(insertWholeStageCodegen(child)))) case p => p.withNewChildren(p.children.map(insertInputAdapter)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 112090640040a..b160b8ac2ed68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -90,7 +90,7 @@ case class AdaptiveSparkPlanExec( // Exchange nodes) after running these rules. private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( ensureRequirements - ) + ) ++ context.session.sessionState.queryStagePrepRules // A list of physical optimizer rules to be applied to a new stage before its execution. These // optimizations should be stage-independent. @@ -100,7 +100,12 @@ case class AdaptiveSparkPlanExec( // The following two rules need to make use of 'CustomShuffleReaderExec.partitionSpecs' // added by `CoalesceShufflePartitions`. So they must be executed after it. OptimizeSkewedJoin(conf), - OptimizeLocalShuffleReader(conf), + OptimizeLocalShuffleReader(conf) + ) + + // A list of physical optimizer rules to be applied right after a new stage is created. The input + // plan to these rules has exchange as its root node. + @transient private val postStageCreationRules = Seq( ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules), CollapseCodegenStages(conf) ) @@ -227,7 +232,8 @@ case class AdaptiveSparkPlanExec( } // Run the final plan when there's no more unfinished stages. - currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) + currentPhysicalPlan = applyPhysicalRules( + result.newPlan, queryStageOptimizerRules ++ postStageCreationRules) isFinalPlan = true executionId.foreach(onUpdatePlan(_, Seq(currentPhysicalPlan))) currentPhysicalPlan @@ -376,10 +382,22 @@ case class AdaptiveSparkPlanExec( private def newQueryStage(e: Exchange): QueryStageExec = { val optimizedPlan = applyPhysicalRules(e.child, queryStageOptimizerRules) val queryStage = e match { - case s: ShuffleExchangeExec => - ShuffleQueryStageExec(currentStageId, s.copy(child = optimizedPlan)) - case b: BroadcastExchangeExec => - BroadcastQueryStageExec(currentStageId, b.copy(child = optimizedPlan)) + case s: ShuffleExchangeLike => + val newShuffle = applyPhysicalRules( + s.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) + if (!newShuffle.isInstanceOf[ShuffleExchangeLike]) { + throw new IllegalStateException( + "Custom columnar rules cannot transform shuffle node to something else.") + } + ShuffleQueryStageExec(currentStageId, newShuffle) + case b: BroadcastExchangeLike => + val newBroadcast = applyPhysicalRules( + b.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) + if (!newBroadcast.isInstanceOf[BroadcastExchangeLike]) { + throw new IllegalStateException( + "Custom columnar rules cannot transform broadcast node to something else.") + } + BroadcastQueryStageExec(currentStageId, newBroadcast) } currentStageId += 1 setLogicalLinkForNewQueryStage(queryStage, e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala index af18ee065aa86..49a4c25fa637f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD @@ -25,8 +24,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -45,6 +45,8 @@ case class CustomShuffleReaderExec private( assert(partitionSpecs.forall(_.isInstanceOf[PartialMapperPartitionSpec])) } + override def supportsColumnar: Boolean = child.supportsColumnar + override def output: Seq[Attribute] = child.output override lazy val outputPartitioning: Partitioning = { // If it is a local shuffle reader with one mapper per task, then the output partitioning is @@ -55,9 +57,9 @@ case class CustomShuffleReaderExec private( partitionSpecs.map(_.asInstanceOf[PartialMapperPartitionSpec].mapIndex).toSet.size == partitionSpecs.length) { child match { - case ShuffleQueryStageExec(_, s: ShuffleExchangeExec) => + case ShuffleQueryStageExec(_, s: ShuffleExchangeLike) => s.child.outputPartitioning - case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeExec)) => + case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeLike)) => s.child.outputPartitioning match { case e: Expression => r.updateAttr(e).asInstanceOf[Partitioning] case other => other @@ -176,18 +178,21 @@ case class CustomShuffleReaderExec private( } } - private lazy val cachedShuffleRDD: RDD[InternalRow] = { + private lazy val shuffleRDD: RDD[_] = { sendDriverMetrics() shuffleStage.map { stage => - new ShuffledRowRDD( - stage.shuffle.shuffleDependency, stage.shuffle.readMetrics, partitionSpecs.toArray) + stage.shuffle.getShuffleRDD(partitionSpecs.toArray) }.getOrElse { throw new IllegalStateException("operating on canonicalized plan") } } override protected def doExecute(): RDD[InternalRow] = { - cachedShuffleRDD + shuffleRDD.asInstanceOf[RDD[InternalRow]] + } + + override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { + shuffleRDD.asInstanceOf[RDD[ColumnarBatch]] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala index ac98342277bc0..bcf9dc1544ce3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.Strategy import org.apache.spark.sql.catalyst.expressions.PredicateHelper import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} -import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, ExtractSingleColumnNullAwareAntiJoin} +import org.apache.spark.sql.catalyst.plans.LeftAnti import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{joins, SparkPlan} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} /** @@ -48,6 +49,11 @@ object LogicalQueryStageStrategy extends Strategy with PredicateHelper { Seq(BroadcastHashJoinExec( leftKeys, rightKeys, joinType, buildSide, condition, planLater(left), planLater(right))) + case j @ ExtractSingleColumnNullAwareAntiJoin(leftKeys, rightKeys) + if isBroadcastStage(j.right) => + Seq(joins.BroadcastHashJoinExec(leftKeys, rightKeys, LeftAnti, BuildRight, + None, planLater(j.left), planLater(j.right), isNullAwareAntiJoin = true)) + case j @ Join(left, right, joinType, condition, _) if isBroadcastStage(left) || isBroadcastStage(right) => val buildSide = if (isBroadcastStage(left)) BuildLeft else BuildRight diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 3620f27058af2..7bb9265e1717a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -78,10 +78,9 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { private def getPartitionSpecs( shuffleStage: ShuffleQueryStageExec, advisoryParallelism: Option[Int]): Seq[ShufflePartitionSpec] = { - val shuffleDep = shuffleStage.shuffle.shuffleDependency - val numReducers = shuffleDep.partitioner.numPartitions + val numMappers = shuffleStage.shuffle.numMappers + val numReducers = shuffleStage.shuffle.numPartitions val expectedParallelism = advisoryParallelism.getOrElse(numReducers) - val numMappers = shuffleDep.rdd.getNumPartitions val splitPoints = if (numMappers == 0) { Seq.empty } else { @@ -141,9 +140,9 @@ object OptimizeLocalShuffleReader { def canUseLocalShuffleReader(plan: SparkPlan): Boolean = plan match { case s: ShuffleQueryStageExec => - s.shuffle.canChangeNumPartitions + s.shuffle.canChangeNumPartitions && s.mapStats.isDefined case CustomShuffleReaderExec(s: ShuffleQueryStageExec, partitionSpecs) => - s.shuffle.canChangeNumPartitions && partitionSpecs.nonEmpty + s.shuffle.canChangeNumPartitions && s.mapStats.isDefined && partitionSpecs.nonEmpty case _ => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index 627f0600f2383..a85b188727ba4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -202,7 +202,7 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { val leftParts = if (isLeftSkew && !isLeftCoalesced) { val reducerId = leftPartSpec.startReducerIndex val skewSpecs = createSkewPartitionSpecs( - left.shuffleStage.shuffle.shuffleDependency.shuffleId, reducerId, leftTargetSize) + left.mapStats.shuffleId, reducerId, leftTargetSize) if (skewSpecs.isDefined) { logDebug(s"Left side partition $partitionIndex " + s"(${FileUtils.byteCountToDisplaySize(leftActualSize)}) is skewed, " + @@ -218,7 +218,7 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { val rightParts = if (isRightSkew && !isRightCoalesced) { val reducerId = rightPartSpec.startReducerIndex val skewSpecs = createSkewPartitionSpecs( - right.shuffleStage.shuffle.shuffleDependency.shuffleId, reducerId, rightTargetSize) + right.mapStats.shuffleId, reducerId, rightTargetSize) if (skewSpecs.isDefined) { logDebug(s"Right side partition $partitionIndex " + s"(${FileUtils.byteCountToDisplaySize(rightActualSize)}) is skewed, " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index 4e83b4344fbf0..0927ef5b0b3c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.ThreadUtils /** @@ -81,17 +82,19 @@ abstract class QueryStageExec extends LeafExecNode { def newReuseInstance(newStageId: Int, newOutput: Seq[Attribute]): QueryStageExec + /** + * Returns the runtime statistics after stage materialization. + */ + def getRuntimeStatistics: Statistics + /** * Compute the statistics of the query stage if executed, otherwise None. */ def computeStats(): Option[Statistics] = resultOption.get().map { _ => - // Metrics `dataSize` are available in both `ShuffleExchangeExec` and `BroadcastExchangeExec`. - val exchange = plan match { - case r: ReusedExchangeExec => r.child - case e: Exchange => e - case _ => throw new IllegalStateException("wrong plan for query stage:\n " + plan.treeString) - } - Statistics(sizeInBytes = exchange.metrics("dataSize").value) + val runtimeStats = getRuntimeStatistics + val dataSize = runtimeStats.sizeInBytes.max(0) + val numOutputRows = runtimeStats.rowCount.map(_.max(0)) + Statistics(dataSize, numOutputRows) } @transient @@ -110,6 +113,8 @@ abstract class QueryStageExec extends LeafExecNode { protected override def doPrepare(): Unit = plan.prepare() protected override def doExecute(): RDD[InternalRow] = plan.execute() + override def supportsColumnar: Boolean = plan.supportsColumnar + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = plan.executeColumnar() override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast() override def doCanonicalize(): SparkPlan = plan.canonicalized @@ -138,15 +143,15 @@ abstract class QueryStageExec extends LeafExecNode { } /** - * A shuffle query stage whose child is a [[ShuffleExchangeExec]] or [[ReusedExchangeExec]]. + * A shuffle query stage whose child is a [[ShuffleExchangeLike]] or [[ReusedExchangeExec]]. */ case class ShuffleQueryStageExec( override val id: Int, override val plan: SparkPlan) extends QueryStageExec { @transient val shuffle = plan match { - case s: ShuffleExchangeExec => s - case ReusedExchangeExec(_, s: ShuffleExchangeExec) => s + case s: ShuffleExchangeLike => s + case ReusedExchangeExec(_, s: ShuffleExchangeLike) => s case _ => throw new IllegalStateException("wrong plan for shuffle stage:\n " + plan.treeString) } @@ -177,22 +182,24 @@ case class ShuffleQueryStageExec( * this method returns None, as there is no map statistics. */ def mapStats: Option[MapOutputStatistics] = { - assert(resultOption.get().isDefined, "ShuffleQueryStageExec should already be ready") + assert(resultOption.get().isDefined, s"${getClass.getSimpleName} should already be ready") val stats = resultOption.get().get.asInstanceOf[MapOutputStatistics] Option(stats) } + + override def getRuntimeStatistics: Statistics = shuffle.runtimeStatistics } /** - * A broadcast query stage whose child is a [[BroadcastExchangeExec]] or [[ReusedExchangeExec]]. + * A broadcast query stage whose child is a [[BroadcastExchangeLike]] or [[ReusedExchangeExec]]. */ case class BroadcastQueryStageExec( override val id: Int, override val plan: SparkPlan) extends QueryStageExec { @transient val broadcast = plan match { - case b: BroadcastExchangeExec => b - case ReusedExchangeExec(_, b: BroadcastExchangeExec) => b + case b: BroadcastExchangeLike => b + case ReusedExchangeExec(_, b: BroadcastExchangeLike) => b case _ => throw new IllegalStateException("wrong plan for broadcast stage:\n " + plan.treeString) } @@ -231,6 +238,8 @@ case class BroadcastQueryStageExec( broadcast.relationFuture.cancel(true) } } + + override def getRuntimeStatistics: Statistics = broadcast.runtimeStatistics } object BroadcastQueryStageExec { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala index 83fdafbadcb60..ed92af6adc186 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala @@ -92,9 +92,9 @@ object ShufflePartitionsUtil extends Logging { var coalescedSize = 0L var i = 0 - def createPartitionSpec(): Unit = { + def createPartitionSpec(forceCreate: Boolean = false): Unit = { // Skip empty inputs, as it is a waste to launch an empty task. - if (coalescedSize > 0) { + if (coalescedSize > 0 || forceCreate) { partitionSpecs += CoalescedPartitionSpec(latestSplitPoint, i) } } @@ -120,7 +120,8 @@ object ShufflePartitionsUtil extends Logging { } i += 1 } - createPartitionSpec() + // Create at least one partition if all partitions are empty. + createPartitionSpec(partitionSpecs.isEmpty) partitionSpecs.toSeq } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala index 67cd720bb5b33..cdc57dbc7dcc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/simpleCosting.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} /** * A simple implementation of [[Cost]], which takes a number of [[Long]] as the cost value. @@ -35,13 +35,13 @@ case class SimpleCost(value: Long) extends Cost { /** * A simple implementation of [[CostEvaluator]], which counts the number of - * [[ShuffleExchangeExec]] nodes in the plan. + * [[ShuffleExchangeLike]] nodes in the plan. */ object SimpleCostEvaluator extends CostEvaluator { override def evaluateCost(plan: SparkPlan): Cost = { val cost = plan.collect { - case s: ShuffleExchangeExec => s + case s: ShuffleExchangeLike => s }.size SimpleCost(cost) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala new file mode 100644 index 0000000000000..22f308f331449 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoin.scala @@ -0,0 +1,177 @@ +/* + * 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.bucketing + +import scala.annotation.tailrec + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.joins.{BaseJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf + +/** + * This rule coalesces one side of the `SortMergeJoin` and `ShuffledHashJoin` + * if the following conditions are met: + * - Two bucketed tables are joined. + * - Join keys match with output partition expressions on their respective sides. + * - The larger bucket number is divisible by the smaller bucket number. + * - COALESCE_BUCKETS_IN_JOIN_ENABLED is set to true. + * - The ratio of the number of buckets is less than the value set in + * COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO. + */ +case class CoalesceBucketsInJoin(conf: SQLConf) extends Rule[SparkPlan] { + private def updateNumCoalescedBucketsInScan( + plan: SparkPlan, + numCoalescedBuckets: Int): SparkPlan = { + plan transformUp { + case f: FileSourceScanExec => + f.copy(optionalNumCoalescedBuckets = Some(numCoalescedBuckets)) + } + } + + private def updateNumCoalescedBuckets( + join: BaseJoinExec, + numLeftBuckets: Int, + numRightBucket: Int, + numCoalescedBuckets: Int): BaseJoinExec = { + if (numCoalescedBuckets != numLeftBuckets) { + val leftCoalescedChild = + updateNumCoalescedBucketsInScan(join.left, numCoalescedBuckets) + join match { + case j: SortMergeJoinExec => j.copy(left = leftCoalescedChild) + case j: ShuffledHashJoinExec => j.copy(left = leftCoalescedChild) + } + } else { + val rightCoalescedChild = + updateNumCoalescedBucketsInScan(join.right, numCoalescedBuckets) + join match { + case j: SortMergeJoinExec => j.copy(right = rightCoalescedChild) + case j: ShuffledHashJoinExec => j.copy(right = rightCoalescedChild) + } + } + } + + private def isCoalesceSHJStreamSide( + join: ShuffledHashJoinExec, + numLeftBuckets: Int, + numRightBucket: Int, + numCoalescedBuckets: Int): Boolean = { + if (numCoalescedBuckets == numLeftBuckets) { + join.buildSide != BuildRight + } else { + join.buildSide != BuildLeft + } + } + + def apply(plan: SparkPlan): SparkPlan = { + if (!conf.coalesceBucketsInJoinEnabled) { + return plan + } + + plan transform { + case ExtractJoinWithBuckets(join, numLeftBuckets, numRightBuckets) + if math.max(numLeftBuckets, numRightBuckets) / math.min(numLeftBuckets, numRightBuckets) <= + conf.coalesceBucketsInJoinMaxBucketRatio => + val numCoalescedBuckets = math.min(numLeftBuckets, numRightBuckets) + join match { + case j: SortMergeJoinExec => + updateNumCoalescedBuckets(j, numLeftBuckets, numRightBuckets, numCoalescedBuckets) + case j: ShuffledHashJoinExec + // Only coalesce the buckets for shuffled hash join stream side, + // to avoid OOM for build side. + if isCoalesceSHJStreamSide(j, numLeftBuckets, numRightBuckets, numCoalescedBuckets) => + updateNumCoalescedBuckets(j, numLeftBuckets, numRightBuckets, numCoalescedBuckets) + case other => other + } + case other => other + } + } +} + +/** + * An extractor that extracts `SortMergeJoinExec` and `ShuffledHashJoin`, + * where both sides of the join have the bucketed tables, + * are consisted of only the scan operation, + * and numbers of buckets are not equal but divisible. + */ +object ExtractJoinWithBuckets { + @tailrec + private def hasScanOperation(plan: SparkPlan): Boolean = plan match { + case f: FilterExec => hasScanOperation(f.child) + case p: ProjectExec => hasScanOperation(p.child) + case _: FileSourceScanExec => true + case _ => false + } + + private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { + plan.collectFirst { + case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && + f.optionalNumCoalescedBuckets.isEmpty => + f.relation.bucketSpec.get + } + } + + /** + * The join keys should match with expressions for output partitioning. Note that + * the ordering does not matter because it will be handled in `EnsureRequirements`. + */ + private def satisfiesOutputPartitioning( + keys: Seq[Expression], + partitioning: Partitioning): Boolean = { + partitioning match { + case HashPartitioning(exprs, _) if exprs.length == keys.length => + exprs.forall(e => keys.exists(_.semanticEquals(e))) + case _ => false + } + } + + private def isApplicable(j: BaseJoinExec): Boolean = { + (j.isInstanceOf[SortMergeJoinExec] || + j.isInstanceOf[ShuffledHashJoinExec]) && + hasScanOperation(j.left) && + hasScanOperation(j.right) && + satisfiesOutputPartitioning(j.leftKeys, j.left.outputPartitioning) && + satisfiesOutputPartitioning(j.rightKeys, j.right.outputPartitioning) + } + + private def isDivisible(numBuckets1: Int, numBuckets2: Int): Boolean = { + val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) + // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller + // number of buckets because bucket id is calculated by modding the total number of buckets. + numBuckets1 != numBuckets2 && large % small == 0 + } + + def unapply(plan: SparkPlan): Option[(BaseJoinExec, Int, Int)] = { + plan match { + case j: BaseJoinExec if isApplicable(j) => + val leftBucket = getBucketSpec(j.left) + val rightBucket = getBucketSpec(j.right) + if (leftBucket.isDefined && rightBucket.isDefined && + isDivisible(leftBucket.get.numBuckets, rightBucket.get.numBuckets)) { + Some(j, leftBucket.get.numBuckets, rightBucket.get.numBuckets) + } else { + None + } + case _ => None + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala deleted file mode 100644 index 3bb0597ecd87c..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoin.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.bucketing - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} -import org.apache.spark.sql.execution.joins.SortMergeJoinExec -import org.apache.spark.sql.internal.SQLConf - -/** - * This rule coalesces one side of the `SortMergeJoin` if the following conditions are met: - * - Two bucketed tables are joined. - * - Join keys match with output partition expressions on their respective sides. - * - The larger bucket number is divisible by the smaller bucket number. - * - COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED is set to true. - * - The ratio of the number of buckets is less than the value set in - * COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO. - */ -case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] { - private def mayCoalesce(numBuckets1: Int, numBuckets2: Int, conf: SQLConf): Option[Int] = { - assert(numBuckets1 != numBuckets2) - val (small, large) = (math.min(numBuckets1, numBuckets2), math.max(numBuckets1, numBuckets2)) - // A bucket can be coalesced only if the bigger number of buckets is divisible by the smaller - // number of buckets because bucket id is calculated by modding the total number of buckets. - if (large % small == 0 && - large / small <= conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO)) { - Some(small) - } else { - None - } - } - - private def updateNumCoalescedBuckets(plan: SparkPlan, numCoalescedBuckets: Int): SparkPlan = { - plan.transformUp { - case f: FileSourceScanExec => - f.copy(optionalNumCoalescedBuckets = Some(numCoalescedBuckets)) - } - } - - def apply(plan: SparkPlan): SparkPlan = { - if (!conf.getConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED)) { - return plan - } - - plan transform { - case ExtractSortMergeJoinWithBuckets(smj, numLeftBuckets, numRightBuckets) - if numLeftBuckets != numRightBuckets => - mayCoalesce(numLeftBuckets, numRightBuckets, conf).map { numCoalescedBuckets => - if (numCoalescedBuckets != numLeftBuckets) { - smj.copy(left = updateNumCoalescedBuckets(smj.left, numCoalescedBuckets)) - } else { - smj.copy(right = updateNumCoalescedBuckets(smj.right, numCoalescedBuckets)) - } - }.getOrElse(smj) - case other => other - } - } -} - -/** - * An extractor that extracts `SortMergeJoinExec` where both sides of the join have the bucketed - * tables and are consisted of only the scan operation. - */ -object ExtractSortMergeJoinWithBuckets { - private def isScanOperation(plan: SparkPlan): Boolean = plan match { - case f: FilterExec => isScanOperation(f.child) - case p: ProjectExec => isScanOperation(p.child) - case _: FileSourceScanExec => true - case _ => false - } - - private def getBucketSpec(plan: SparkPlan): Option[BucketSpec] = { - plan.collectFirst { - case f: FileSourceScanExec if f.relation.bucketSpec.nonEmpty && - f.optionalNumCoalescedBuckets.isEmpty => - f.relation.bucketSpec.get - } - } - - /** - * The join keys should match with expressions for output partitioning. Note that - * the ordering does not matter because it will be handled in `EnsureRequirements`. - */ - private def satisfiesOutputPartitioning( - keys: Seq[Expression], - partitioning: Partitioning): Boolean = { - partitioning match { - case HashPartitioning(exprs, _) if exprs.length == keys.length => - exprs.forall(e => keys.exists(_.semanticEquals(e))) - case _ => false - } - } - - private def isApplicable(s: SortMergeJoinExec): Boolean = { - isScanOperation(s.left) && - isScanOperation(s.right) && - satisfiesOutputPartitioning(s.leftKeys, s.left.outputPartitioning) && - satisfiesOutputPartitioning(s.rightKeys, s.right.outputPartitioning) - } - - def unapply(plan: SparkPlan): Option[(SortMergeJoinExec, Int, Int)] = { - plan match { - case s: SortMergeJoinExec if isApplicable(s) => - val leftBucket = getBucketSpec(s.left) - val rightBucket = getBucketSpec(s.right) - if (leftBucket.isDefined && rightBucket.isDefined) { - Some(s, leftBucket.get.numBuckets, rightBucket.get.numBuckets) - } else { - None - } - case _ => None - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index 20ecc57c49e75..45557bfbada6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializable { +class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() @@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes) } -private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { +class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { val (forAttribute: AttributeMap[ColumnStatisticsSchema], schema: Seq[AttributeReference]) = { val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) (AttributeMap(allStats), allStats.flatMap(_._2.schema)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index bd2d06665a910..eb0663830dd67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.CalendarInterval * An Iterator to walk through the InternalRows from a CachedBatch */ abstract class ColumnarIterator extends Iterator[InternalRow] { - def initialize(input: Iterator[CachedBatch], columnTypes: Array[DataType], + def initialize(input: Iterator[DefaultCachedBatch], columnTypes: Array[DataType], columnIndexes: Array[Int]): Unit } @@ -203,7 +203,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera return false; } - ${classOf[CachedBatch].getName} batch = (${classOf[CachedBatch].getName}) input.next(); + ${classOf[DefaultCachedBatch].getName} batch = + (${classOf[DefaultCachedBatch].getName}) input.next(); currentRow = 0; numRowsInBatch = batch.numRows(); for (int i = 0; i < columnIndexes.length; i ++) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index cf9f3ddeb42a2..07411c0d3803c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -19,33 +19,189 @@ package org.apache.spark.sql.execution.columnar import org.apache.commons.lang3.StringUtils +import org.apache.spark.TaskContext import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer} +import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.LongAccumulator - +import org.apache.spark.util.{LongAccumulator, Utils} /** - * CachedBatch is a cached batch of rows. + * The default implementation of CachedBatch. * * @param numRows The total number of rows in this batch * @param buffers The buffers for serialized columns * @param stats The stat of columns */ -private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) +case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) + extends SimpleMetricsCachedBatch + +/** + * The default implementation of CachedBatchSerializer. + */ +class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false + + override def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = + throw new IllegalStateException("Columnar input is not supported") + + override def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + val batchSize = conf.columnBatchSize + val useCompression = conf.useCompression + convertForCacheInternal(input, schema, batchSize, useCompression) + } + + def convertForCacheInternal( + input: RDD[InternalRow], + output: Seq[Attribute], + batchSize: Int, + useCompression: Boolean): RDD[CachedBatch] = { + input.mapPartitionsInternal { rowIterator => + new Iterator[DefaultCachedBatch] { + def next(): DefaultCachedBatch = { + val columnBuilders = output.map { attribute => + ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) + }.toArray + + var rowCount = 0 + var totalSize = 0L + while (rowIterator.hasNext && rowCount < batchSize + && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { + val row = rowIterator.next() + + // Added for SPARK-6082. This assertion can be useful for scenarios when something + // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM + // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat + // hard to decipher. + assert( + row.numFields == columnBuilders.length, + s"Row column number mismatch, expected ${output.size} columns, " + + s"but got ${row.numFields}." + + s"\nRow content: $row") + + var i = 0 + totalSize = 0 + while (i < row.numFields) { + columnBuilders(i).appendFrom(row, i) + totalSize += columnBuilders(i).columnStats.sizeInBytes + i += 1 + } + rowCount += 1 + } + + val stats = InternalRow.fromSeq( + columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) + DefaultCachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, stats) + } + + def hasNext: Boolean = rowIterator.hasNext + } + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f => + f.dataType match { + // More types can be supported, but this is to match the original implementation that + // only supported primitive types "for ease of review" + case BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType => true + case _ => false + }) + + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Option(Seq.fill(attributes.length)( + if (!conf.offHeapColumnVectorEnabled) { + classOf[OnHeapColumnVector].getName + } else { + classOf[OffHeapColumnVector].getName + } + )) + + override def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled + val outputSchema = StructType.fromAttributes(selectedAttributes) + val columnIndices = + selectedAttributes.map(a => cacheAttributes.map(o => o.exprId).indexOf(a.exprId)).toArray + + def createAndDecompressColumn(cb: CachedBatch): ColumnarBatch = { + val cachedColumnarBatch = cb.asInstanceOf[DefaultCachedBatch] + val rowCount = cachedColumnarBatch.numRows + val taskContext = Option(TaskContext.get()) + val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { + OnHeapColumnVector.allocateColumns(rowCount, outputSchema) + } else { + OffHeapColumnVector.allocateColumns(rowCount, outputSchema) + } + val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) + columnarBatch.setNumRows(rowCount) + + for (i <- selectedAttributes.indices) { + ColumnAccessor.decompress( + cachedColumnarBatch.buffers(columnIndices(i)), + columnarBatch.column(i).asInstanceOf[WritableColumnVector], + outputSchema.fields(i).dataType, rowCount) + } + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) + columnarBatch + } + + input.map(createAndDecompressColumn) + } + + override def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + // Find the ordinals and data types of the requested columns. + val (requestedColumnIndices, requestedColumnDataTypes) = + selectedAttributes.map { a => + cacheAttributes.map(_.exprId).indexOf(a.exprId) -> a.dataType + }.unzip + + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }.toArray + + input.mapPartitionsInternal { cachedBatchIterator => + val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + columnarIterator.initialize(cachedBatchIterator.asInstanceOf[Iterator[DefaultCachedBatch]], + columnTypes, + requestedColumnIndices.toArray) + columnarIterator + } + } +} +private[sql] case class CachedRDDBuilder( - useCompression: Boolean, - batchSize: Int, + serializer: CachedBatchSerializer, storageLevel: StorageLevel, @transient cachedPlan: SparkPlan, tableName: Option[String]) { @@ -85,54 +241,24 @@ case class CachedRDDBuilder( } private def buildBuffers(): RDD[CachedBatch] = { - val output = cachedPlan.output - val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator => - new Iterator[CachedBatch] { - def next(): CachedBatch = { - val columnBuilders = output.map { attribute => - ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) - }.toArray - - var rowCount = 0 - var totalSize = 0L - while (rowIterator.hasNext && rowCount < batchSize - && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { - val row = rowIterator.next() - - // Added for SPARK-6082. This assertion can be useful for scenarios when something - // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM - // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat - // hard to decipher. - assert( - row.numFields == columnBuilders.length, - s"Row column number mismatch, expected ${output.size} columns, " + - s"but got ${row.numFields}." + - s"\nRow content: $row") - - var i = 0 - totalSize = 0 - while (i < row.numFields) { - columnBuilders(i).appendFrom(row, i) - totalSize += columnBuilders(i).columnStats.sizeInBytes - i += 1 - } - rowCount += 1 - } - - sizeInBytesStats.add(totalSize) - rowCountStats.add(rowCount) - - val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - } - - def hasNext: Boolean = rowIterator.hasNext - } + val cb = if (cachedPlan.supportsColumnar) { + serializer.convertColumnarBatchToCachedBatch( + cachedPlan.executeColumnar(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } else { + serializer.convertInternalRowToCachedBatch( + cachedPlan.execute(), + cachedPlan.output, + storageLevel, + cachedPlan.conf) + } + val cached = cb.map { batch => + sizeInBytesStats.add(batch.sizeInBytes) + rowCountStats.add(batch.numRows) + batch }.persist(storageLevel) - cached.setName(cachedName) cached } @@ -140,22 +266,74 @@ case class CachedRDDBuilder( object InMemoryRelation { + private[this] var ser: Option[CachedBatchSerializer] = None + private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized { + if (ser.isEmpty) { + val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER) + val serClass = Utils.classForName(serName) + val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer] + ser = Some(instance) + } + ser.get + } + + /* Visible for testing */ + private[columnar] def clearSerializer(): Unit = synchronized { ser = None } + + def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match { + case gen: WholeStageCodegenExec => gen.child match { + case c2r: ColumnarToRowTransition => c2r.child match { + case ia: InputAdapter => ia.child + case _ => plan + } + case _ => plan + } + case c2r: ColumnarToRowTransition => // This matches when whole stage code gen is disabled. + c2r.child + case _ => plan + } + def apply( - useCompression: Boolean, - batchSize: Int, + storageLevel: StorageLevel, + qe: QueryExecution, + tableName: Option[String]): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val serializer = getSerializer(optimizedPlan.conf) + val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) { + convertToColumnarIfPossible(qe.executedPlan) + } else { + qe.executedPlan + } + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) + val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) + relation.statsOfPlanToCache = optimizedPlan.stats + relation + } + + /** + * This API is intended only to be used for testing. + */ + def apply( + serializer: CachedBatchSerializer, storageLevel: StorageLevel, child: SparkPlan, tableName: Option[String], optimizedPlan: LogicalPlan): InMemoryRelation = { - val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName) + val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName) val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation } - def apply(cacheBuilder: CachedRDDBuilder, optimizedPlan: LogicalPlan): InMemoryRelation = { + def apply(cacheBuilder: CachedRDDBuilder, qe: QueryExecution): InMemoryRelation = { + val optimizedPlan = qe.optimizedPlan + val newBuilder = if (cacheBuilder.serializer.supportsColumnarInput(optimizedPlan.output)) { + cacheBuilder.copy(cachedPlan = convertToColumnarIfPossible(qe.executedPlan)) + } else { + cacheBuilder.copy(cachedPlan = qe.executedPlan) + } val relation = new InMemoryRelation( - cacheBuilder.cachedPlan.output, cacheBuilder, optimizedPlan.outputOrdering) + newBuilder.cachedPlan.output, newBuilder, optimizedPlan.outputOrdering) relation.statsOfPlanToCache = optimizedPlan.stats relation } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index f03c2586048bd..e4194562b7a5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,19 +17,15 @@ package org.apache.spark.sql.execution.columnar -import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.columnar.CachedBatch import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.execution.vectorized._ -import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} - +import org.apache.spark.sql.vectorized.ColumnarBatch case class InMemoryTableScanExec( attributes: Seq[Attribute], @@ -57,68 +53,29 @@ case class InMemoryTableScanExec( relation = relation.canonicalized.asInstanceOf[InMemoryRelation]) override def vectorTypes: Option[Seq[String]] = - Option(Seq.fill(attributes.length)( - if (!conf.offHeapColumnVectorEnabled) { - classOf[OnHeapColumnVector].getName - } else { - classOf[OffHeapColumnVector].getName - } - )) + relation.cacheBuilder.serializer.vectorTypes(attributes, conf) /** * If true, get data from ColumnVector in ColumnarBatch, which are generally faster. * If false, get data from UnsafeRow build from CachedBatch */ override val supportsColumnar: Boolean = { - // In the initial implementation, for ease of review - // support only primitive data types and # of fields is less than wholeStageMaxNumFields - conf.cacheVectorizedReaderEnabled && relation.schema.fields.forall(f => f.dataType match { - case BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType => true - case _ => false - }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) - } - - private val columnIndices = - attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray - - private val relationSchema = relation.schema.toArray - - private lazy val columnarBatchSchema = new StructType(columnIndices.map(i => relationSchema(i))) - - private def createAndDecompressColumn( - cachedColumnarBatch: CachedBatch, - offHeapColumnVectorEnabled: Boolean): ColumnarBatch = { - val rowCount = cachedColumnarBatch.numRows - val taskContext = Option(TaskContext.get()) - val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) { - OnHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } else { - OffHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema) - } - val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]]) - columnarBatch.setNumRows(rowCount) - - for (i <- attributes.indices) { - ColumnAccessor.decompress( - cachedColumnarBatch.buffers(columnIndices(i)), - columnarBatch.column(i).asInstanceOf[WritableColumnVector], - columnarBatchSchema.fields(i).dataType, rowCount) - } - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close())) - columnarBatch + conf.cacheVectorizedReaderEnabled && + !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) && + relation.cacheBuilder.serializer.supportsColumnarOutput(relation.schema) } private lazy val columnarInputRDD: RDD[ColumnarBatch] = { val numOutputRows = longMetric("numOutputRows") val buffers = filteredCachedBatches() - val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled - buffers - .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) - .map { buffer => - numOutputRows += buffer.numRows() - buffer - } + relation.cacheBuilder.serializer.convertCachedBatchToColumnarBatch( + buffers, + relation.output, + attributes, + conf).map { cb => + numOutputRows += cb.numRows() + cb + } } private lazy val inputRDD: RDD[InternalRow] = { @@ -130,35 +87,24 @@ case class InMemoryTableScanExec( val numOutputRows = longMetric("numOutputRows") // Using these variables here to avoid serialization of entire objects (if referenced // directly) within the map Partitions closure. - val relOutput: AttributeSeq = relation.output - - filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => - // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = - attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType - }.unzip - - // update SQL metrics - val withMetrics = cachedBatchIterator.map { batch => - if (enableAccumulatorsForTest) { - readBatches.add(1) + val relOutput = relation.output + val serializer = relation.cacheBuilder.serializer + + // update SQL metrics + val withMetrics = + filteredCachedBatches().mapPartitionsInternal { iter => + if (enableAccumulatorsForTest && iter.hasNext) { + readPartitions.add(1) + } + iter.map { batch => + if (enableAccumulatorsForTest) { + readBatches.add(1) + } + numOutputRows += batch.numRows + batch } - numOutputRows += batch.numRows - batch - } - - val columnTypes = requestedColumnDataTypes.map { - case udt: UserDefinedType[_] => udt.sqlType - case other => other - }.toArray - val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) - if (enableAccumulatorsForTest && columnarIterator.hasNext) { - readPartitions.add(1) } - columnarIterator - } + serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf) } override def output: Seq[Attribute] = attributes @@ -186,114 +132,6 @@ case class InMemoryTableScanExec( override def outputOrdering: Seq[SortOrder] = relation.cachedPlan.outputOrdering.map(updateAttribute(_).asInstanceOf[SortOrder]) - // Keeps relation's partition statistics because we don't serialize relation. - private val stats = relation.partitionStatistics - private def statsFor(a: Attribute) = stats.forAttribute(a) - - // Currently, only use statistics from atomic types except binary type only. - private object ExtractableLiteral { - def unapply(expr: Expression): Option[Literal] = expr match { - case lit: Literal => lit.dataType match { - case BinaryType => None - case _: AtomicType => Some(lit) - case _ => None - } - case _ => None - } - } - - // Returned filter predicate should return false iff it is impossible for the input expression - // to evaluate to `true` based on statistics collected about this partition batch. - @transient lazy val buildFilter: PartialFunction[Expression, Expression] = { - case And(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => - (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) - - case Or(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => - buildFilter(lhs) || buildFilter(rhs) - - case EqualTo(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualTo(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - - case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l - case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound - - case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound <= l - case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - l <= statsFor(a).upperBound - - case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound - case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l - - case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) => - l <= statsFor(a).upperBound - case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) => - statsFor(a).lowerBound <= l - - case IsNull(a: Attribute) => statsFor(a).nullCount > 0 - case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 - - case In(a: AttributeReference, list: Seq[Expression]) - if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => - list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && - l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) - - // This is an example to explain how it works, imagine that the id column stored as follows: - // __________________________________________ - // | Partition ID | lowerBound | upperBound | - // |--------------|------------|------------| - // | p1 | '1' | '9' | - // | p2 | '10' | '19' | - // | p3 | '20' | '29' | - // | p4 | '30' | '39' | - // | p5 | '40' | '49' | - // |______________|____________|____________| - // - // A filter: df.filter($"id".startsWith("2")). - // In this case it substr lowerBound and upperBound: - // ________________________________________________________________________________________ - // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | - // |--------------|-----------------------------------|-----------------------------------| - // | p1 | '1' | '9' | - // | p2 | '1' | '1' | - // | p3 | '2' | '2' | - // | p4 | '3' | '3' | - // | p5 | '4' | '4' | - // |______________|___________________________________|___________________________________| - // - // We can see that we only need to read p1 and p3. - case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => - statsFor(a).lowerBound.substr(0, Length(l)) <= l && - l <= statsFor(a).upperBound.substr(0, Length(l)) - } - - lazy val partitionFilters: Seq[Expression] = { - predicates.flatMap { p => - val filter = buildFilter.lift(p) - val boundFilter = - filter.map( - BindReferences.bindReference( - _, - stats.schema, - allowFailures = true)) - - boundFilter.foreach(_ => - filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) - - // If the filter can't be resolved then we are missing required statistics. - boundFilter.filter(_.resolved) - } - } - lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes @@ -303,37 +141,13 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning private def filteredCachedBatches(): RDD[CachedBatch] = { - // Using these variables here to avoid serialization of entire objects (if referenced directly) - // within the map Partitions closure. - val schema = stats.schema - val schemaIndex = schema.zipWithIndex val buffers = relation.cacheBuilder.cachedColumnBuffers - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = Predicate.create( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - - // Do partition batch pruning if enabled - if (inMemoryPartitionPruningEnabled) { - cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter.eval(cachedBatch.stats)) { - logDebug { - val statsString = schemaIndex.map { case (a, i) => - val value = cachedBatch.stats.get(i, a.dataType) - s"${a.name}: $value" - }.mkString(", ") - s"Skipping partition based on stats $statsString" - } - false - } else { - true - } - } - } else { - cachedBatchIterator - } + if (inMemoryPartitionPruningEnabled) { + val filterFunc = relation.cacheBuilder.serializer.buildFilter(predicates, relation.output) + buffers.mapPartitionsWithIndexInternal(filterFunc) + } else { + buffers } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 3dc1d52697714..fd89e361fe3d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -166,15 +166,22 @@ object SetCommand { * via [[SetCommand]] will get reset to default value. Command that runs * {{{ * reset; + * reset spark.sql.session.timeZone; * }}} */ -case object ResetCommand extends RunnableCommand with IgnoreCachedData { +case class ResetCommand(config: Option[String]) extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { - val conf = sparkSession.sessionState.conf - conf.clear() - sparkSession.sparkContext.conf.getAll.foreach { case (k, v) => - conf.setConfString(k, v) + val defaults = sparkSession.sparkContext.conf + config match { + case Some(key) => + sparkSession.conf.unset(key) + defaults.getOption(key).foreach(sparkSession.conf.set(key, _)) + case None => + sparkSession.sessionState.conf.clear() + defaults.getAll.foreach { case (k, v) => + sparkSession.sessionState.conf.setConfString(k, v) + } } Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 6fdc7f4a58195..252d188ff8fe2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -236,6 +236,45 @@ case class ShowFunctionsCommand( } } + +/** + * A command for users to refresh the persistent function. + * The syntax of using this command in SQL is: + * {{{ + * REFRESH FUNCTION functionName + * }}} + */ +case class RefreshFunctionCommand( + databaseName: Option[String], + functionName: String) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + if (FunctionRegistry.builtin.functionExists(FunctionIdentifier(functionName))) { + throw new AnalysisException(s"Cannot refresh builtin function $functionName") + } + if (catalog.isTemporaryFunction(FunctionIdentifier(functionName, databaseName))) { + throw new AnalysisException(s"Cannot refresh temporary function $functionName") + } + + val identifier = FunctionIdentifier( + functionName, Some(databaseName.getOrElse(catalog.getCurrentDatabase))) + // we only refresh the permanent function. + if (catalog.isPersistentFunction(identifier)) { + // register overwrite function. + val func = catalog.getFunctionMetadata(identifier) + catalog.registerFunction(func, true) + } else { + // clear cached function and throw exception + catalog.unregisterFunction(identifier) + throw new NoSuchFunctionException(identifier.database.get, identifier.funcName) + } + + Seq.empty[Row] + } +} + object FunctionsCommand { // operators that do not have corresponding functions. // They should be handled `DescribeFunctionCommand`, `ShowFunctionsCommand` diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index db564485be883..36e5eb33e1bae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -421,18 +421,18 @@ case class DataSource( relation match { case hs: HadoopFsRelation => - SchemaUtils.checkColumnNameDuplication( - hs.dataSchema.map(_.name), + SchemaUtils.checkSchemaColumnNameDuplication( + hs.dataSchema, "in the data schema", equality) - SchemaUtils.checkColumnNameDuplication( - hs.partitionSchema.map(_.name), + SchemaUtils.checkSchemaColumnNameDuplication( + hs.partitionSchema, "in the partition schema", equality) DataSourceUtils.verifySchema(hs.fileFormat, hs.dataSchema) case _ => - SchemaUtils.checkColumnNameDuplication( - relation.schema.map(_.name), + SchemaUtils.checkSchemaColumnNameDuplication( + relation.schema, "in the data schema", equality) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala index aea27bd4c4d7f..b5d800f02862e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ @@ -44,7 +45,9 @@ object FileStatusCache { session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) { if (sharedCache == null) { sharedCache = new SharedInMemoryCache( - session.sqlContext.conf.filesourcePartitionFileCacheSize) + session.sqlContext.conf.filesourcePartitionFileCacheSize, + session.sqlContext.conf.metadataCacheTTL + ) } sharedCache.createForNewClient() } else { @@ -89,7 +92,7 @@ abstract class FileStatusCache { * * @param maxSizeInBytes max allowable cache size before entries start getting evicted */ -private class SharedInMemoryCache(maxSizeInBytes: Long) extends Logging { +private class SharedInMemoryCache(maxSizeInBytes: Long, cacheTTL: Long) extends Logging { // Opaque object that uniquely identifies a shared cache user private type ClientId = Object @@ -129,11 +132,17 @@ private class SharedInMemoryCache(maxSizeInBytes: Long) extends Logging { } } } - CacheBuilder.newBuilder() + + var builder = CacheBuilder.newBuilder() .weigher(weigher) .removalListener(removalListener) .maximumWeight(maxSizeInBytes / weightScale) - .build[(ClientId, Path), Array[FileStatus]]() + + if (cacheTTL > 0) { + builder = builder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS) + } + + builder.build[(ClientId, Path), Array[FileStatus]]() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 2e09c729529a6..5341e22f5e670 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -56,14 +56,17 @@ abstract class PartitioningAwareFileIndex( protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] - protected lazy val pathGlobFilter = parameters.get("pathGlobFilter").map(new GlobFilter(_)) + private val caseInsensitiveMap = CaseInsensitiveMap(parameters) + + protected lazy val pathGlobFilter: Option[GlobFilter] = + caseInsensitiveMap.get("pathGlobFilter").map(new GlobFilter(_)) protected def matchGlobPattern(file: FileStatus): Boolean = { pathGlobFilter.forall(_.accept(file.getPath)) } - protected lazy val recursiveFileLookup = { - parameters.getOrElse("recursiveFileLookup", "false").toBoolean + protected lazy val recursiveFileLookup: Boolean = { + caseInsensitiveMap.getOrElse("recursiveFileLookup", "false").toBoolean } override def listFiles( @@ -215,7 +218,7 @@ abstract class PartitioningAwareFileIndex( * and the returned DataFrame will have the column of `something`. */ private def basePaths: Set[Path] = { - parameters.get(BASE_PATH_PARAM).map(new Path(_)) match { + caseInsensitiveMap.get(BASE_PATH_PARAM).map(new Path(_)) match { case Some(userDefinedBasePath) => val fs = userDefinedBasePath.getFileSystem(hadoopConf) if (!fs.isDirectory(userDefinedBasePath)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 576a826faf894..0c56e7675da6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -56,8 +56,10 @@ private[sql] object PruneFileSourcePartitions val (partitionFilters, dataFilters) = normalizedFilters.partition(f => f.references.subsetOf(partitionSet) ) + val extraPartitionFilter = + dataFilters.flatMap(extractPredicatesWithinOutputSet(_, partitionSet)) - (ExpressionSet(partitionFilters), dataFilters) + (ExpressionSet(partitionFilters ++ extraPartitionFilter), dataFilters) } private def rebuildPhysicalOperation( @@ -88,10 +90,8 @@ private[sql] object PruneFileSourcePartitions _, _)) if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => - val predicates = CNFWithGroupExpressionsByReference(filters.reduceLeft(And)) - val finalPredicates = if (predicates.nonEmpty) predicates else filters val (partitionKeyFilters, _) = getPartitionKeyFiltersAndDataFilters( - fsRelation.sparkSession, logicalRelation, partitionSchema, finalPredicates, + fsRelation.sparkSession, logicalRelation, partitionSchema, filters, logicalRelation.output) if (partitionKeyFilters.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index e7456f9c8ed0b..314012feef4c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -59,7 +59,7 @@ class JdbcRelationProvider extends CreatableRelationProvider } else { // Otherwise, do not truncate the table, instead drop and recreate it dropTable(conn, options.table, options) - createTable(conn, df, options) + createTable(conn, options.table, df.schema, isCaseSensitive, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } @@ -78,7 +78,7 @@ class JdbcRelationProvider extends CreatableRelationProvider // Therefore, it is okay to do nothing here and then just return the relation below. } } else { - createTable(conn, df, options) + createTable(conn, options.table, df.schema, isCaseSensitive, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index db4715ef068b6..5831c35c7e301 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Connection, Driver, DriverManager, JDBCType, PreparedStatement, ResultSet, ResultSetMetaData, SQLException} +import java.sql.{Connection, Driver, DriverManager, JDBCType, PreparedStatement, ResultSet, ResultSetMetaData, SQLException, SQLFeatureNotSupportedException} import java.util.Locale import scala.collection.JavaConverters._ @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.connector.catalog.TableChange import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.types._ @@ -94,13 +95,7 @@ object JdbcUtils extends Logging { * Drops a table from the JDBC database. */ def dropTable(conn: Connection, table: String, options: JDBCOptions): Unit = { - val statement = conn.createStatement - try { - statement.setQueryTimeout(options.queryTimeout) - statement.executeUpdate(s"DROP TABLE $table") - } finally { - statement.close() - } + executeStatement(conn, options, s"DROP TABLE $table") } /** @@ -184,7 +179,7 @@ object JdbcUtils extends Logging { } } - private def getJdbcType(dt: DataType, dialect: JdbcDialect): JdbcType = { + def getJdbcType(dt: DataType, dialect: JdbcDialect): JdbcType = { dialect.getJDBCType(dt).orElse(getCommonJDBCType(dt)).getOrElse( throw new IllegalArgumentException(s"Can't get JDBC type for ${dt.catalogString}")) } @@ -746,15 +741,16 @@ object JdbcUtils extends Logging { * Compute the schema string for this RDD. */ def schemaString( - df: DataFrame, + schema: StructType, + caseSensitive: Boolean, url: String, createTableColumnTypes: Option[String] = None): String = { val sb = new StringBuilder() val dialect = JdbcDialects.get(url) val userSpecifiedColTypesMap = createTableColumnTypes - .map(parseUserSpecifiedCreateTableColumnTypes(df, _)) + .map(parseUserSpecifiedCreateTableColumnTypes(schema, caseSensitive, _)) .getOrElse(Map.empty[String, String]) - df.schema.fields.foreach { field => + schema.fields.foreach { field => val name = dialect.quoteIdentifier(field.name) val typ = userSpecifiedColTypesMap .getOrElse(field.name, getJdbcType(field.dataType, dialect).databaseTypeDefinition) @@ -770,7 +766,8 @@ object JdbcUtils extends Logging { * use in-place of the default data type. */ private def parseUserSpecifiedCreateTableColumnTypes( - df: DataFrame, + schema: StructType, + caseSensitive: Boolean, createTableColumnTypes: String): Map[String, String] = { def typeName(f: StructField): String = { // char/varchar gets translated to string type. Real data type specified by the user @@ -783,7 +780,11 @@ object JdbcUtils extends Logging { } val userSchema = CatalystSqlParser.parseTableSchema(createTableColumnTypes) - val nameEquality = df.sparkSession.sessionState.conf.resolver + val nameEquality = if (caseSensitive) { + org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + } else { + org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution + } // checks duplicate columns in the user specified column types. SchemaUtils.checkColumnNameDuplication( @@ -791,16 +792,15 @@ object JdbcUtils extends Logging { // checks if user specified column names exist in the DataFrame schema userSchema.fieldNames.foreach { col => - df.schema.find(f => nameEquality(f.name, col)).getOrElse { + schema.find(f => nameEquality(f.name, col)).getOrElse { throw new AnalysisException( s"createTableColumnTypes option column $col not found in schema " + - df.schema.catalogString) + schema.catalogString) } } val userSchemaMap = userSchema.fields.map(f => f.name -> typeName(f)).toMap - val isCaseSensitive = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - if (isCaseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) + if (caseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) } /** @@ -814,8 +814,10 @@ object JdbcUtils extends Logging { if (null != customSchema && customSchema.nonEmpty) { val userSchema = CatalystSqlParser.parseTableSchema(customSchema) - SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "in the customSchema option value", nameEquality) + SchemaUtils.checkSchemaColumnNameDuplication( + userSchema, + "in the customSchema option value", + nameEquality) // This is resolved by names, use the custom filed dataType to replace the default dataType. val newSchema = tableSchema.map { col => @@ -865,17 +867,71 @@ object JdbcUtils extends Logging { */ def createTable( conn: Connection, - df: DataFrame, + tableName: String, + schema: StructType, + caseSensitive: Boolean, options: JdbcOptionsInWrite): Unit = { val strSchema = schemaString( - df, options.url, options.createTableColumnTypes) - val table = options.table + schema, caseSensitive, options.url, options.createTableColumnTypes) val createTableOptions = options.createTableOptions // Create the table if the table does not exist. // To allow certain options to append when create a new table, which can be // table_options or partition_options. // E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" - val sql = s"CREATE TABLE $table ($strSchema) $createTableOptions" + val sql = s"CREATE TABLE $tableName ($strSchema) $createTableOptions" + executeStatement(conn, options, sql) + } + + /** + * Rename a table from the JDBC database. + */ + def renameTable( + conn: Connection, + oldTable: String, + newTable: String, + options: JDBCOptions): Unit = { + val dialect = JdbcDialects.get(options.url) + executeStatement(conn, options, dialect.renameTable(oldTable, newTable)) + } + + /** + * Update a table from the JDBC database. + */ + def alterTable( + conn: Connection, + tableName: String, + changes: Seq[TableChange], + options: JDBCOptions): Unit = { + val dialect = JdbcDialects.get(options.url) + if (changes.length == 1) { + executeStatement(conn, options, dialect.alterTable(tableName, changes)(0)) + } else { + val metadata = conn.getMetaData + if (!metadata.supportsTransactions) { + throw new SQLFeatureNotSupportedException("The target JDBC server does not support " + + "transaction and can only support ALTER TABLE with a single action.") + } else { + conn.setAutoCommit(false) + val statement = conn.createStatement + try { + statement.setQueryTimeout(options.queryTimeout) + for (sql <- dialect.alterTable(tableName, changes)) { + statement.executeUpdate(sql) + } + conn.commit() + } catch { + case e: Exception => + if (conn != null) conn.rollback() + throw e + } finally { + statement.close() + conn.setAutoCommit(true) + } + } + } + } + + private def executeStatement(conn: Connection, options: JDBCOptions, sql: String): Unit = { val statement = conn.createStatement try { statement.setQueryTimeout(options.queryTimeout) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 95a63c3d1e2d9..e0fa4584185e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -125,7 +125,11 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { } (file: PartitionedFile) => { - val parser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) + val parser = new JacksonParser( + actualSchema, + parsedOptions, + allowArrayAsStructs = true, + filters) JsonDataSource(parsedOptions).readFile( broadcastedHadoopConf.value.value, file, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index fd791ce7c5e19..4dff1ec7ebfb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -164,8 +164,6 @@ class OrcFileFormat val enableVectorizedReader = supportBatch(sparkSession, resultSchema) val capacity = sqlConf.orcVectorizedReaderBatchSize - val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) - OrcConf.MAPRED_INPUT_SCHEMA.setString(hadoopConf, resultSchemaString) OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(hadoopConf, sqlConf.caseSensitiveAnalysis) val broadcastedConf = @@ -179,16 +177,18 @@ class OrcFileFormat val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val requestedColIdsOrEmptyFile = + val resultedColPruneInfo = Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, requiredSchema, reader, conf) } - if (requestedColIdsOrEmptyFile.isEmpty) { + if (resultedColPruneInfo.isEmpty) { Iterator.empty } else { - val requestedColIds = requestedColIdsOrEmptyFile.get + val (requestedColIds, canPruneCols) = resultedColPruneInfo.get + val resultSchemaString = OrcUtils.orcResultSchemaString(canPruneCols, + dataSchema, resultSchema, partitionSchema, conf) assert(requestedColIds.length == requiredSchema.length, "[BUG] requested column IDs do not match required schema") val taskConf = new Configuration(conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index d274bcd0edd2c..072e670081d1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.orc.{OrcFile, Reader, TypeDescription, Writer} +import org.apache.orc.{OrcConf, OrcFile, Reader, TypeDescription, Writer} import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} import org.apache.spark.deploy.SparkHadoopUtil @@ -116,15 +116,17 @@ object OrcUtils extends Logging { } /** - * Returns the requested column ids from the given ORC file. Column id can be -1, which means the - * requested column doesn't exist in the ORC file. Returns None if the given ORC file is empty. + * @return Returns the combination of requested column ids from the given ORC file and + * boolean flag to find if the pruneCols is allowed or not. Requested Column id can be + * -1, which means the requested column doesn't exist in the ORC file. Returns None + * if the given ORC file is empty. */ def requestedColumnIds( isCaseSensitive: Boolean, dataSchema: StructType, requiredSchema: StructType, reader: Reader, - conf: Configuration): Option[Array[Int]] = { + conf: Configuration): Option[(Array[Int], Boolean)] = { val orcFieldNames = reader.getSchema.getFieldNames.asScala if (orcFieldNames.isEmpty) { // SPARK-8501: Some old empty ORC files always have an empty schema stored in their footer. @@ -136,6 +138,10 @@ object OrcUtils extends Logging { assert(orcFieldNames.length <= dataSchema.length, "The given data schema " + s"${dataSchema.catalogString} has less fields than the actual ORC physical schema, " + "no idea which columns were dropped, fail to read.") + // for ORC file written by Hive, no field names + // in the physical schema, there is a need to send the + // entire dataSchema instead of required schema. + // So pruneCols is not done in this case Some(requiredSchema.fieldNames.map { name => val index = dataSchema.fieldIndex(name) if (index < orcFieldNames.length) { @@ -143,7 +149,7 @@ object OrcUtils extends Logging { } else { -1 } - }) + }, false) } else { if (isCaseSensitive) { Some(requiredSchema.fieldNames.zipWithIndex.map { case (name, idx) => @@ -152,7 +158,7 @@ object OrcUtils extends Logging { } else { -1 } - }) + }, true) } else { // Do case-insensitive resolution only if in case-insensitive mode val caseInsensitiveOrcFieldMap = orcFieldNames.groupBy(_.toLowerCase(Locale.ROOT)) @@ -170,7 +176,7 @@ object OrcUtils extends Logging { idx } }.getOrElse(-1) - }) + }, true) } } } @@ -199,4 +205,31 @@ object OrcUtils extends Logging { s"map<${orcTypeDescriptionString(m.keyType)},${orcTypeDescriptionString(m.valueType)}>" case _ => dt.catalogString } + + /** + * Returns the result schema to read from ORC file. In addition, It sets + * the schema string to 'orc.mapred.input.schema' so ORC reader can use later. + * + * @param canPruneCols Flag to decide whether pruned cols schema is send to resultSchema + * or to send the entire dataSchema to resultSchema. + * @param dataSchema Schema of the orc files. + * @param resultSchema Result data schema created after pruning cols. + * @param partitionSchema Schema of partitions. + * @param conf Hadoop Configuration. + * @return Returns the result schema as string. + */ + def orcResultSchemaString( + canPruneCols: Boolean, + dataSchema: StructType, + resultSchema: StructType, + partitionSchema: StructType, + conf: Configuration): String = { + val resultSchemaString = if (canPruneCols) { + OrcUtils.orcTypeDescriptionString(resultSchema) + } else { + OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ partitionSchema.fields)) + } + OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) + resultSchemaString + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 59dc3ae56bf25..7bd05f12873f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -79,7 +79,7 @@ abstract class FileTable( override lazy val schema: StructType = { val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - SchemaUtils.checkColumnNameDuplication(dataSchema.fieldNames, + SchemaUtils.checkSchemaColumnNameDuplication(dataSchema, "in the data schema", caseSensitive) dataSchema.foreach { field => if (!supportsDataType(field.dataType)) { @@ -88,7 +88,7 @@ abstract class FileTable( } } val partitionSchema = fileIndex.partitionSchema - SchemaUtils.checkColumnNameDuplication(partitionSchema.fieldNames, + SchemaUtils.checkSchemaColumnNameDuplication(partitionSchema, "in the partition schema", caseSensitive) val partitionNameSet: Set[String] = partitionSchema.fields.map(PartitioningUtils.getColName(_, caseSensitive)).toSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index df3f231f7d0ef..6dda1d4aaf37e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -126,11 +126,18 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) val comment = properties.get(TableCatalog.PROP_COMMENT) val owner = properties.getOrElse(TableCatalog.PROP_OWNER, catalogTable.owner) + val location = properties.get(TableCatalog.PROP_LOCATION).map(CatalogUtils.stringToURI) + val storage = if (location.isDefined) { + catalogTable.storage.copy(locationUri = location) + } else { + catalogTable.storage + } try { catalog.alterTable( - catalogTable - .copy(properties = properties, schema = schema, owner = owner, comment = comment)) + catalogTable.copy( + properties = properties, schema = schema, owner = owner, comment = comment, + storage = storage)) } catch { case _: NoSuchTableException => throw new NoSuchTableException(ident) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index 81a234e254000..f7a79bf31948e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.csv.CSVFilters +import org.apache.spark.sql.catalyst.StructFilters import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder @@ -49,7 +49,7 @@ case class CSVScanBuilder( override def pushFilters(filters: Array[Filter]): Array[Filter] = { if (sparkSession.sessionState.conf.csvFilterPushDown) { - _pushedFilters = CSVFilters.pushedFilters(filters, dataSchema) + _pushedFilters = StructFilters.pushedFilters(filters, dataSchema) } filters } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala new file mode 100644 index 0000000000000..c91723d1331c5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.jdbc + +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.sql.types.StructType + +// TODO (SPARK-32396): Implement the `SupportsRead` interface +// TODO (SPARK-32410): Implement the `SupportsWrite` interface +case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOptions) + extends Table { + assert(ident.namespace().length == 1) + + override def name(): String = ident.toString + + override def capabilities(): java.util.Set[TableCapability] = { + val capabilities = new java.util.HashSet[TableCapability] + capabilities.add(BATCH_READ) + capabilities + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala new file mode 100644 index 0000000000000..0138014a8e21e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.jdbc + +import java.sql.{Connection, SQLException} + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException} +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog, TableChange} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JDBCRDD, JdbcUtils} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JDBCTableCatalog extends TableCatalog with Logging { + + private var catalogName: String = null + private var options: JDBCOptions = _ + private var dialect: JdbcDialect = _ + + override def name(): String = { + require(catalogName != null, "The JDBC table catalog is not initialed") + catalogName + } + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + assert(catalogName == null, "The JDBC table catalog is already initialed") + catalogName = name + + val map = options.asCaseSensitiveMap().asScala.toMap + // The `JDBCOptions` checks the existence of the table option. This is required by JDBC v1, but + // JDBC V2 only knows the table option when loading a table. Here we put a table option with a + // fake value, so that it can pass the check of `JDBCOptions`. + this.options = new JDBCOptions(map + (JDBCOptions.JDBC_TABLE_NAME -> "__invalid_dbtable")) + dialect = JdbcDialects.get(this.options.url) + } + + override def listTables(namespace: Array[String]): Array[Identifier] = { + checkNamespace(namespace) + withConnection { conn => + val schemaPattern = if (namespace.length == 1) namespace.head else null + val rs = conn.getMetaData + .getTables(null, schemaPattern, "%", Array("TABLE")); + new Iterator[Identifier] { + def hasNext = rs.next() + def next = Identifier.of(namespace, rs.getString("TABLE_NAME")) + }.toArray + } + } + + override def tableExists(ident: Identifier): Boolean = { + checkNamespace(ident.namespace()) + val writeOptions = new JdbcOptionsInWrite( + options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) + withConnection(JdbcUtils.tableExists(_, writeOptions)) + } + + override def dropTable(ident: Identifier): Boolean = { + checkNamespace(ident.namespace()) + withConnection { conn => + try { + JdbcUtils.dropTable(conn, getTableName(ident), options) + true + } catch { + case _: SQLException => false + } + } + } + + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { + checkNamespace(oldIdent.namespace()) + withConnection { conn => + JdbcUtils.renameTable(conn, getTableName(oldIdent), getTableName(newIdent), options) + } + } + + override def loadTable(ident: Identifier): Table = { + checkNamespace(ident.namespace()) + val optionsWithTableName = new JDBCOptions( + options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) + try { + val schema = JDBCRDD.resolveTable(optionsWithTableName) + JDBCTable(ident, schema, optionsWithTableName) + } catch { + case _: SQLException => throw new NoSuchTableException(ident) + } + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: java.util.Map[String, String]): Table = { + checkNamespace(ident.namespace()) + if (partitions.nonEmpty) { + throw new UnsupportedOperationException("Cannot create JDBC table with partition") + } + // TODO (SPARK-32405): Apply table options while creating tables in JDBC Table Catalog + if (!properties.isEmpty) { + logWarning("Cannot create JDBC table with properties, these properties will be " + + "ignored: " + properties.asScala.map { case (k, v) => s"$k=$v" }.mkString("[", ", ", "]")) + } + + val writeOptions = new JdbcOptionsInWrite( + options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) + val caseSensitive = SQLConf.get.caseSensitiveAnalysis + withConnection { conn => + JdbcUtils.createTable(conn, getTableName(ident), schema, caseSensitive, writeOptions) + } + + JDBCTable(ident, schema, writeOptions) + } + + override def alterTable(ident: Identifier, changes: TableChange*): Table = { + checkNamespace(ident.namespace()) + withConnection { conn => + JdbcUtils.alterTable(conn, getTableName(ident), changes, options) + loadTable(ident) + } + } + + private def checkNamespace(namespace: Array[String]): Unit = { + // In JDBC there is no nested database/schema + if (namespace.length > 1) { + throw new NoSuchNamespaceException(namespace) + } + } + + private def withConnection[T](f: Connection => T): T = { + val conn = JdbcUtils.createConnectionFactory(options)() + try { + f(conn) + } finally { + conn.close() + } + } + + private def getTableName(ident: Identifier): String = { + (ident.namespace() :+ ident.name()).map(dialect.quoteIdentifier).mkString(".") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala index 698423948f916..9737803b597a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -36,6 +37,7 @@ import org.apache.spark.util.SerializableConfiguration * @param readDataSchema Required schema of JSON files. * @param partitionSchema Schema of partitions. * @param parsedOptions Options for parsing JSON files. + * @param filters The filters pushed down to JSON datasource. */ case class JsonPartitionReaderFactory( sqlConf: SQLConf, @@ -43,12 +45,17 @@ case class JsonPartitionReaderFactory( dataSchema: StructType, readDataSchema: StructType, partitionSchema: StructType, - parsedOptions: JSONOptionsInRead) extends FilePartitionReaderFactory { + parsedOptions: JSONOptionsInRead, + filters: Seq[Filter]) extends FilePartitionReaderFactory { override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { val actualSchema = StructType(readDataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val parser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) + val parser = new JacksonParser( + actualSchema, + parsedOptions, + allowArrayAsStructs = true, + filters) val iter = JsonDataSource(parsedOptions).readFile( broadcastedConf.value.value, partitionedFile, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala index 75231625676ff..7ad106627a083 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -39,6 +40,7 @@ case class JsonScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter], partitionFilters: Seq[Expression] = Seq.empty, dataFilters: Seq[Expression] = Seq.empty) extends TextBasedFileScan(sparkSession, options) { @@ -86,7 +88,7 @@ case class JsonScan( // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. JsonPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + dataSchema, readDataSchema, readPartitionSchema, parsedOptions, pushedFilters) } override def withFilters( @@ -94,10 +96,14 @@ case class JsonScan( this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { - case j: JsonScan => super.equals(j) && dataSchema == j.dataSchema && options == j.options - + case j: JsonScan => super.equals(j) && dataSchema == j.dataSchema && options == j.options && + equivalentFilters(pushedFilters, j.pushedFilters) case _ => false } override def hashCode(): Int = super.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala index be53b1b1676f1..cf1204566ddbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution.datasources.v2.json import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.catalyst.StructFilters +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -29,8 +31,26 @@ class JsonScanBuilder ( schema: StructType, dataSchema: StructType, options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { override def build(): Scan = { - JsonScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + JsonScan( + sparkSession, + fileIndex, + dataSchema, + readDataSchema(), + readPartitionSchema(), + options, + pushedFilters()) } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (sparkSession.sessionState.conf.jsonFilterPushDown) { + _pushedFilters = StructFilters.pushedFilters(filters, dataSchema) + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 03d58fdcb7f67..7f25f7bd135f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -66,24 +66,24 @@ case class OrcPartitionReaderFactory( override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { val conf = broadcastedConf.value.value - val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) - OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, isCaseSensitive) val filePath = new Path(new URI(file.filePath)) val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val requestedColIdsOrEmptyFile = + val resultedColPruneInfo = Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) } - if (requestedColIdsOrEmptyFile.isEmpty) { + if (resultedColPruneInfo.isEmpty) { new EmptyPartitionReader[InternalRow] } else { - val requestedColIds = requestedColIdsOrEmptyFile.get + val (requestedColIds, canPruneCols) = resultedColPruneInfo.get + val resultSchemaString = OrcUtils.orcResultSchemaString(canPruneCols, + dataSchema, resultSchema, partitionSchema, conf) assert(requestedColIds.length == readDataSchema.length, "[BUG] requested column IDs do not match required schema") @@ -112,24 +112,25 @@ case class OrcPartitionReaderFactory( override def buildColumnarReader(file: PartitionedFile): PartitionReader[ColumnarBatch] = { val conf = broadcastedConf.value.value - val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) - OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, isCaseSensitive) val filePath = new Path(new URI(file.filePath)) val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val requestedColIdsOrEmptyFile = + val resultedColPruneInfo = Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) } - if (requestedColIdsOrEmptyFile.isEmpty) { + if (resultedColPruneInfo.isEmpty) { new EmptyPartitionReader } else { - val requestedColIds = requestedColIdsOrEmptyFile.get ++ Array.fill(partitionSchema.length)(-1) + val (requestedDataColIds, canPruneCols) = resultedColPruneInfo.get + val resultSchemaString = OrcUtils.orcResultSchemaString(canPruneCols, + dataSchema, resultSchema, partitionSchema, conf) + val requestedColIds = requestedDataColIds ++ Array.fill(partitionSchema.length)(-1) assert(requestedColIds.length == resultSchema.length, "[BUG] requested column IDs do not match required schema") val taskConf = new Configuration(conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala index cfc653a23840d..098576d72f540 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala @@ -59,9 +59,9 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession) // the first to be applied (apart from `InsertAdaptiveSparkPlan`). val canReuseExchange = SQLConf.get.exchangeReuseEnabled && buildKeys.nonEmpty && plan.find { - case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _) => + case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _, _) => left.sameResult(sparkPlan) - case BroadcastHashJoinExec(_, _, _, BuildRight, _, _, right) => + case BroadcastHashJoinExec(_, _, _, BuildRight, _, _, right, _) => right.sameResult(sparkPlan) case _ => false }.isDefined diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index d35bbe9b8adc0..6d8d37022ea42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -29,6 +29,7 @@ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning} import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.joins.HashedRelation @@ -37,16 +38,43 @@ import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.util.{SparkFatalException, ThreadUtils} +/** + * Common trait for all broadcast exchange implementations to facilitate pattern matching. + */ +trait BroadcastExchangeLike extends Exchange { + + /** + * The broadcast job group ID + */ + def runId: UUID = UUID.randomUUID + + /** + * The asynchronous job that prepares the broadcast relation. + */ + def relationFuture: Future[broadcast.Broadcast[Any]] + + /** + * For registering callbacks on `relationFuture`. + * Note that calling this method may not start the execution of broadcast job. + */ + def completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] + + /** + * Returns the runtime statistics after broadcast materialization. + */ + def runtimeStatistics: Statistics +} + /** * A [[BroadcastExchangeExec]] collects, transforms and finally broadcasts the result of * a transformed SparkPlan. */ case class BroadcastExchangeExec( mode: BroadcastMode, - child: SparkPlan) extends Exchange { + child: SparkPlan) extends BroadcastExchangeLike { import BroadcastExchangeExec._ - private[sql] val runId: UUID = UUID.randomUUID + override val runId: UUID = UUID.randomUUID override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), @@ -60,21 +88,23 @@ case class BroadcastExchangeExec( BroadcastExchangeExec(mode.canonicalized, child.canonicalized) } + override def runtimeStatistics: Statistics = { + val dataSize = metrics("dataSize").value + Statistics(dataSize) + } + @transient private lazy val promise = Promise[broadcast.Broadcast[Any]]() - /** - * For registering callbacks on `relationFuture`. - * Note that calling this field will not start the execution of broadcast job. - */ @transient - lazy val completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] = promise.future + override lazy val completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] = + promise.future @transient private val timeout: Long = SQLConf.get.broadcastTimeout @transient - private[sql] lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + override lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { SQLExecution.withThreadLocalCaptured[broadcast.Broadcast[Any]]( sqlContext.sparkSession, BroadcastExchangeExec.executionContext) { try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index b06742e8470c7..6af4b098bee2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -30,8 +30,9 @@ import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProces import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Divide, Literal, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering +import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} @@ -40,13 +41,54 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.util.MutablePair import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator} +/** + * Common trait for all shuffle exchange implementations to facilitate pattern matching. + */ +trait ShuffleExchangeLike extends Exchange { + + /** + * Returns the number of mappers of this shuffle. + */ + def numMappers: Int + + /** + * Returns the shuffle partition number. + */ + def numPartitions: Int + + /** + * Returns whether the shuffle partition number can be changed. + */ + def canChangeNumPartitions: Boolean + + /** + * The asynchronous job that materializes the shuffle. + */ + def mapOutputStatisticsFuture: Future[MapOutputStatistics] + + /** + * Returns the shuffle RDD with specified partition specs. + */ + def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] + + /** + * Returns the runtime statistics after shuffle materialization. + */ + def runtimeStatistics: Statistics +} + /** * Performs a shuffle that will result in the desired partitioning. */ case class ShuffleExchangeExec( override val outputPartitioning: Partitioning, child: SparkPlan, - canChangeNumPartitions: Boolean = true) extends Exchange { + noUserSpecifiedNumPartition: Boolean = true) extends ShuffleExchangeLike { + + // If users specify the num partitions via APIs like `repartition`, we shouldn't change it. + // For `SinglePartition`, it requires exactly one partition and we can't change it either. + override def canChangeNumPartitions: Boolean = + noUserSpecifiedNumPartition && outputPartitioning != SinglePartition private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) @@ -64,7 +106,7 @@ case class ShuffleExchangeExec( @transient lazy val inputRDD: RDD[InternalRow] = child.execute() // 'mapOutputStatisticsFuture' is only needed when enable AQE. - @transient lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { + @transient override lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { if (inputRDD.getNumPartitions == 0) { Future.successful(null) } else { @@ -72,6 +114,20 @@ case class ShuffleExchangeExec( } } + override def numMappers: Int = shuffleDependency.rdd.getNumPartitions + + override def numPartitions: Int = shuffleDependency.partitioner.numPartitions + + override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[InternalRow] = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionSpecs) + } + + override def runtimeStatistics: Statistics = { + val dataSize = metrics("dataSize").value + val rowCount = metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN).value + Statistics(dataSize, Some(rowCount)) + } + /** * A [[ShuffleDependency]] that will partition rows of its child based on * the partitioning scheme defined in `newPartitioning`. Those partitions of diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 707ed1402d1ae..e4935c8c72228 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -17,19 +17,19 @@ package org.apache.spark.sql.execution.joins +import scala.collection.mutable + import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, HashPartitioning, Partitioning, PartitioningCollection, UnspecifiedDistribution} import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.{BooleanType, LongType} /** * Performs an inner hash join of two child relations. When the output RDD of this operator is @@ -44,14 +44,23 @@ case class BroadcastHashJoinExec( buildSide: BuildSide, condition: Option[Expression], left: SparkPlan, - right: SparkPlan) + right: SparkPlan, + isNullAwareAntiJoin: Boolean = false) extends HashJoin with CodegenSupport { + if (isNullAwareAntiJoin) { + require(leftKeys.length == 1, "leftKeys length should be 1") + require(rightKeys.length == 1, "rightKeys length should be 1") + require(joinType == LeftAnti, "joinType must be LeftAnti.") + require(buildSide == BuildRight, "buildSide must be BuildRight.") + require(condition.isEmpty, "null aware anti join optimize condition should be empty.") + } + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { - val mode = HashedRelationBroadcastMode(buildKeys) + val mode = HashedRelationBroadcastMode(buildBoundKeys, isNullAwareAntiJoin) buildSide match { case BuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil @@ -60,14 +69,108 @@ case class BroadcastHashJoinExec( } } + override lazy val outputPartitioning: Partitioning = { + joinType match { + case _: InnerLike if sqlContext.conf.broadcastHashJoinOutputPartitioningExpandLimit > 0 => + streamedPlan.outputPartitioning match { + case h: HashPartitioning => expandOutputPartitioning(h) + case c: PartitioningCollection => expandOutputPartitioning(c) + case other => other + } + case _ => streamedPlan.outputPartitioning + } + } + + // An one-to-many mapping from a streamed key to build keys. + private lazy val streamedKeyToBuildKeyMapping = { + val mapping = mutable.Map.empty[Expression, Seq[Expression]] + streamedKeys.zip(buildKeys).foreach { + case (streamedKey, buildKey) => + val key = streamedKey.canonicalized + mapping.get(key) match { + case Some(v) => mapping.put(key, v :+ buildKey) + case None => mapping.put(key, Seq(buildKey)) + } + } + mapping.toMap + } + + // Expands the given partitioning collection recursively. + private def expandOutputPartitioning( + partitioning: PartitioningCollection): PartitioningCollection = { + PartitioningCollection(partitioning.partitionings.flatMap { + case h: HashPartitioning => expandOutputPartitioning(h).partitionings + case c: PartitioningCollection => Seq(expandOutputPartitioning(c)) + case other => Seq(other) + }) + } + + // Expands the given hash partitioning by substituting streamed keys with build keys. + // For example, if the expressions for the given partitioning are Seq("a", "b", "c") + // where the streamed keys are Seq("b", "c") and the build keys are Seq("x", "y"), + // the expanded partitioning will have the following expressions: + // Seq("a", "b", "c"), Seq("a", "b", "y"), Seq("a", "x", "c"), Seq("a", "x", "y"). + // The expanded expressions are returned as PartitioningCollection. + private def expandOutputPartitioning(partitioning: HashPartitioning): PartitioningCollection = { + val maxNumCombinations = sqlContext.conf.broadcastHashJoinOutputPartitioningExpandLimit + var currentNumCombinations = 0 + + def generateExprCombinations( + current: Seq[Expression], + accumulated: Seq[Expression]): Seq[Seq[Expression]] = { + if (currentNumCombinations >= maxNumCombinations) { + Nil + } else if (current.isEmpty) { + currentNumCombinations += 1 + Seq(accumulated) + } else { + val buildKeysOpt = streamedKeyToBuildKeyMapping.get(current.head.canonicalized) + generateExprCombinations(current.tail, accumulated :+ current.head) ++ + buildKeysOpt.map(_.flatMap(b => generateExprCombinations(current.tail, accumulated :+ b))) + .getOrElse(Nil) + } + } + + PartitioningCollection( + generateExprCombinations(partitioning.expressions, Nil) + .map(HashPartitioning(_, partitioning.numPartitions))) + } + protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]() - streamedPlan.execute().mapPartitions { streamedIter => - val hashed = broadcastRelation.value.asReadOnlyCopy() - TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) - join(streamedIter, hashed, numOutputRows) + if (isNullAwareAntiJoin) { + streamedPlan.execute().mapPartitionsInternal { streamedIter => + val hashed = broadcastRelation.value.asReadOnlyCopy() + TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) + if (hashed == EmptyHashedRelation) { + streamedIter + } else if (hashed == EmptyHashedRelationWithAllNullKeys) { + Iterator.empty + } else { + val keyGenerator = UnsafeProjection.create( + BindReferences.bindReferences[Expression]( + leftKeys, + AttributeSeq(left.output)) + ) + streamedIter.filter(row => { + val lookupKey: UnsafeRow = keyGenerator(row) + if (lookupKey.anyNull()) { + false + } else { + // Anti Join: Drop the row on the streamed side if it is a match on the build + hashed.get(lookupKey) == null + } + }) + } + } + } else { + streamedPlan.execute().mapPartitions { streamedIter => + val hashed = broadcastRelation.value.asReadOnlyCopy() + TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) + join(streamedIter, hashed, numOutputRows) + } } } @@ -92,23 +195,6 @@ case class BroadcastHashJoinExec( override def needCopyResult: Boolean = streamedPlan.asInstanceOf[CodegenSupport].needCopyResult || multipleOutputForOneInput - override def doProduce(ctx: CodegenContext): String = { - streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this) - } - - override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { - joinType match { - case _: InnerLike => codegenInner(ctx, input) - case LeftOuter | RightOuter => codegenOuter(ctx, input) - case LeftSemi => codegenSemi(ctx, input) - case LeftAnti => codegenAnti(ctx, input) - case j: ExistenceJoin => codegenExistence(ctx, input) - case x => - throw new IllegalArgumentException( - s"BroadcastHashJoin should not take $x as the JoinType") - } - } - /** * Returns a tuple of Broadcast of HashedRelation and the variable name for it. */ @@ -127,377 +213,55 @@ case class BroadcastHashJoinExec( (broadcastRelation, relationTerm) } - /** - * Returns the code for generating join key for stream side, and expression of whether the key - * has any null in it or not. - */ - private def genStreamSideJoinKey( - ctx: CodegenContext, - input: Seq[ExprCode]): (ExprCode, String) = { - ctx.currentVars = input - if (streamedKeys.length == 1 && streamedKeys.head.dataType == LongType) { - // generate the join key as Long - val ev = streamedKeys.head.genCode(ctx) - (ev, ev.isNull) - } else { - // generate the join key as UnsafeRow - val ev = GenerateUnsafeProjection.createCode(ctx, streamedKeys) - (ev, s"${ev.value}.anyNull()") - } - } - - /** - * Generates the code for variable of build side. - */ - private def genBuildSideVars(ctx: CodegenContext, matched: String): Seq[ExprCode] = { - ctx.currentVars = null - ctx.INPUT_ROW = matched - buildPlan.output.zipWithIndex.map { case (a, i) => - val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx) - if (joinType.isInstanceOf[InnerLike]) { - ev - } else { - // the variables are needed even there is no matched rows - val isNull = ctx.freshName("isNull") - val value = ctx.freshName("value") - val javaType = CodeGenerator.javaType(a.dataType) - val code = code""" - |boolean $isNull = true; - |$javaType $value = ${CodeGenerator.defaultValue(a.dataType)}; - |if ($matched != null) { - | ${ev.code} - | $isNull = ${ev.isNull}; - | $value = ${ev.value}; - |} - """.stripMargin - ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType)) - } - } - } - - /** - * Generate the (non-equi) condition used to filter joined rows. This is used in Inner, Left Semi - * and Left Anti joins. - */ - private def getJoinCondition( - ctx: CodegenContext, - input: Seq[ExprCode]): (String, String, Seq[ExprCode]) = { - val matched = ctx.freshName("matched") - val buildVars = genBuildSideVars(ctx, matched) - val checkCondition = if (condition.isDefined) { - val expr = condition.get - // evaluate the variables from build side that used by condition - val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) - // filter the output via condition - ctx.currentVars = input ++ buildVars - val ev = - BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) - val skipRow = s"${ev.isNull} || !${ev.value}" - s""" - |$eval - |${ev.code} - |if (!($skipRow)) - """.stripMargin - } else { - "" - } - (matched, checkCondition, buildVars) - } - - /** - * Generates the code for Inner join. - */ - private def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - - val resultVars = buildSide match { - case BuildLeft => buildVars ++ input - case BuildRight => input ++ buildVars - } - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |if ($matched != null) { - | $checkCondition { - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - |} - """.stripMargin - - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |if ($matches != null) { - | while ($matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition { - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - | } - |} - """.stripMargin - } - } - - /** - * Generates the code for left or right outer join. - */ - private def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val matched = ctx.freshName("matched") - val buildVars = genBuildSideVars(ctx, matched) - val numOutput = metricTerm(ctx, "numOutputRows") - - // filter the output via condition - val conditionPassed = ctx.freshName("conditionPassed") - val checkCondition = if (condition.isDefined) { - val expr = condition.get - // evaluate the variables from build side that used by condition - val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) - ctx.currentVars = input ++ buildVars - val ev = - BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) - s""" - |boolean $conditionPassed = true; - |${eval.trim} - |if ($matched != null) { - | ${ev.code} - | $conditionPassed = !${ev.isNull} && ${ev.value}; - |} - """.stripMargin - } else { - s"final boolean $conditionPassed = true;" - } - - val resultVars = buildSide match { - case BuildLeft => buildVars ++ input - case BuildRight => input ++ buildVars - } - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |${checkCondition.trim} - |if (!$conditionPassed) { - | $matched = null; - | // reset the variables those are already evaluated. - | ${buildVars.filter(_.code.isEmpty).map(v => s"${v.isNull} = true;").mkString("\n")} - |} - |$numOutput.add(1); - |${consume(ctx, resultVars)} - """.stripMargin - - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - val found = ctx.freshName("found") - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |boolean $found = false; - |// the last iteration of this loop is to emit an empty row if there is no matched rows. - |while ($matches != null && $matches.hasNext() || !$found) { - | UnsafeRow $matched = $matches != null && $matches.hasNext() ? - | (UnsafeRow) $matches.next() : null; - | ${checkCondition.trim} - | if ($conditionPassed) { - | $found = true; - | $numOutput.add(1); - | ${consume(ctx, resultVars)} - | } - |} - """.stripMargin - } - } - - /** - * Generates the code for left semi join. - */ - private def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = { + protected override def prepareRelation(ctx: CodegenContext): (String, Boolean) = { val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val (matched, checkCondition, _) = getJoinCondition(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |if ($matched != null) { - | $checkCondition { - | $numOutput.add(1); - | ${consume(ctx, input)} - | } - |} - """.stripMargin - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - val found = ctx.freshName("found") - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |if ($matches != null) { - | boolean $found = false; - | while (!$found && $matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition { - | $found = true; - | } - | } - | if ($found) { - | $numOutput.add(1); - | ${consume(ctx, input)} - | } - |} - """.stripMargin - } + (relationTerm, broadcastRelation.value.keyIsUnique) } /** * Generates the code for anti join. + * Handles NULL-aware anti join (NAAJ) separately here. */ - private def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val uniqueKeyCodePath = broadcastRelation.value.keyIsUnique - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val (matched, checkCondition, _) = getJoinCondition(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - - if (uniqueKeyCodePath) { - val found = ctx.freshName("found") - s""" - |boolean $found = false; - |// generate join key for stream side - |${keyEv.code} - |// Check if the key has nulls. - |if (!($anyNull)) { - | // Check if the HashedRelation exists. - | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - | if ($matched != null) { - | // Evaluate the condition. - | $checkCondition { - | $found = true; - | } - | } - |} - |if (!$found) { - | $numOutput.add(1); - | ${consume(ctx, input)} - |} - """.stripMargin - } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - val found = ctx.freshName("found") - s""" - |boolean $found = false; - |// generate join key for stream side - |${keyEv.code} - |// Check if the key has nulls. - |if (!($anyNull)) { - | // Check if the HashedRelation exists. - | $iteratorCls $matches = ($iteratorCls)$relationTerm.get(${keyEv.value}); - | if ($matches != null) { - | // Evaluate the condition. - | while (!$found && $matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition { - | $found = true; - | } - | } - | } - |} - |if (!$found) { - | $numOutput.add(1); - | ${consume(ctx, input)} - |} - """.stripMargin - } - } + protected override def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { + if (isNullAwareAntiJoin) { + val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, _, _) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") - /** - * Generates the code for existence join. - */ - private def codegenExistence(ctx: CodegenContext, input: Seq[ExprCode]): String = { - val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) - val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) - val numOutput = metricTerm(ctx, "numOutputRows") - val existsVar = ctx.freshName("exists") - - val matched = ctx.freshName("matched") - val buildVars = genBuildSideVars(ctx, matched) - val checkCondition = if (condition.isDefined) { - val expr = condition.get - // evaluate the variables from build side that used by condition - val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) - // filter the output via condition - ctx.currentVars = input ++ buildVars - val ev = - BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) - s""" - |$eval - |${ev.code} - |$existsVar = !${ev.isNull} && ${ev.value}; - """.stripMargin - } else { - s"$existsVar = true;" - } - - val resultVar = input ++ Seq(ExprCode.forNonNullValue( - JavaCode.variable(existsVar, BooleanType))) - if (broadcastRelation.value.keyIsUnique) { - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashedRelation - |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); - |boolean $existsVar = false; - |if ($matched != null) { - | $checkCondition - |} - |$numOutput.add(1); - |${consume(ctx, resultVar)} - """.stripMargin + if (broadcastRelation.value == EmptyHashedRelation) { + s""" + |// If the right side is empty, NAAJ simply returns the left side. + |$numOutput.add(1); + |${consume(ctx, input)} + """.stripMargin + } else if (broadcastRelation.value == EmptyHashedRelationWithAllNullKeys) { + s""" + |// If the right side contains any all-null key, NAAJ simply returns Nothing. + """.stripMargin + } else { + val found = ctx.freshName("found") + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |if ($anyNull) { + | $found = true; + |} else { + | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + | if ($matched != null) { + | $found = true; + | } + |} + | + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } } else { - val matches = ctx.freshName("matches") - val iteratorCls = classOf[Iterator[UnsafeRow]].getName - s""" - |// generate join key for stream side - |${keyEv.code} - |// find matches from HashRelation - |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); - |boolean $existsVar = false; - |if ($matches != null) { - | while (!$existsVar && $matches.hasNext()) { - | UnsafeRow $matched = (UnsafeRow) $matches.next(); - | $checkCondition - | } - |} - |$numOutput.add(1); - |${consume(ctx, resultVar)} - """.stripMargin + super.codegenAnti(ctx, input) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index c7c3e1672f034..1c6504b141890 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -20,14 +20,16 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{ExplainUtils, RowIterator} +import org.apache.spark.sql.execution.{CodegenSupport, ExplainUtils, RowIterator} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.types.{IntegralType, LongType} +import org.apache.spark.sql.types.{BooleanType, IntegralType, LongType} -trait HashJoin extends BaseJoinExec { +trait HashJoin extends BaseJoinExec with CodegenSupport { def buildSide: BuildSide override def simpleStringWithNodeId(): String = { @@ -52,7 +54,41 @@ trait HashJoin extends BaseJoinExec { } } - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning + override def outputPartitioning: Partitioning = buildSide match { + case BuildLeft => + joinType match { + case _: InnerLike | RightOuter => right.outputPartitioning + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building left side") + } + case BuildRight => + joinType match { + case _: InnerLike | LeftOuter | LeftSemi | LeftAnti | _: ExistenceJoin => + left.outputPartitioning + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building right side") + } + } + + override def outputOrdering: Seq[SortOrder] = buildSide match { + case BuildLeft => + joinType match { + case _: InnerLike | RightOuter => right.outputOrdering + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building left side") + } + case BuildRight => + joinType match { + case _: InnerLike | LeftOuter | LeftSemi | LeftAnti | _: ExistenceJoin => + left.outputOrdering + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building right side") + } + } protected lazy val (buildPlan, streamedPlan) = buildSide match { case BuildLeft => (left, right) @@ -62,21 +98,30 @@ trait HashJoin extends BaseJoinExec { protected lazy val (buildKeys, streamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") - val lkeys = bindReferences(HashJoin.rewriteKeyExpr(leftKeys), left.output) - val rkeys = bindReferences(HashJoin.rewriteKeyExpr(rightKeys), right.output) buildSide match { - case BuildLeft => (lkeys, rkeys) - case BuildRight => (rkeys, lkeys) + case BuildLeft => (leftKeys, rightKeys) + case BuildRight => (rightKeys, leftKeys) } } + @transient private lazy val (buildOutput, streamedOutput) = { + buildSide match { + case BuildLeft => (left.output, right.output) + case BuildRight => (right.output, left.output) + } + } + @transient protected lazy val buildBoundKeys = + bindReferences(HashJoin.rewriteKeyExpr(buildKeys), buildOutput) + + @transient protected lazy val streamedBoundKeys = + bindReferences(HashJoin.rewriteKeyExpr(streamedKeys), streamedOutput) protected def buildSideKeyGenerator(): Projection = - UnsafeProjection.create(buildKeys) + UnsafeProjection.create(buildBoundKeys) protected def streamSideKeyGenerator(): UnsafeProjection = - UnsafeProjection.create(streamedKeys) + UnsafeProjection.create(streamedBoundKeys) @transient private[this] lazy val boundCondition = if (condition.isDefined) { Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ @@ -99,47 +144,73 @@ trait HashJoin extends BaseJoinExec { hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinRow = new JoinedRow val joinKeys = streamSideKeyGenerator() - streamIter.flatMap { srow => - joinRow.withLeft(srow) - val matches = hashedRelation.get(joinKeys(srow)) - if (matches != null) { - matches.map(joinRow.withRight(_)).filter(boundCondition) - } else { - Seq.empty + + if (hashedRelation.keyIsUnique) { + streamIter.flatMap { srow => + joinRow.withLeft(srow) + val matched = hashedRelation.getValue(joinKeys(srow)) + if (matched != null) { + Some(joinRow.withRight(matched)).filter(boundCondition) + } else { + None + } + } + } else { + streamIter.flatMap { srow => + joinRow.withLeft(srow) + val matches = hashedRelation.get(joinKeys(srow)) + if (matches != null) { + matches.map(joinRow.withRight).filter(boundCondition) + } else { + Seq.empty + } } } } private def outerJoin( streamedIter: Iterator[InternalRow], - hashedRelation: HashedRelation): Iterator[InternalRow] = { + hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinedRow = new JoinedRow() val keyGenerator = streamSideKeyGenerator() val nullRow = new GenericInternalRow(buildPlan.output.length) - streamedIter.flatMap { currentRow => - val rowKey = keyGenerator(currentRow) - joinedRow.withLeft(currentRow) - val buildIter = hashedRelation.get(rowKey) - new RowIterator { - private var found = false - override def advanceNext(): Boolean = { - while (buildIter != null && buildIter.hasNext) { - val nextBuildRow = buildIter.next() - if (boundCondition(joinedRow.withRight(nextBuildRow))) { + if (hashedRelation.keyIsUnique) { + streamedIter.map { currentRow => + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + val matched = hashedRelation.getValue(rowKey) + if (matched != null && boundCondition(joinedRow.withRight(matched))) { + joinedRow + } else { + joinedRow.withRight(nullRow) + } + } + } else { + streamedIter.flatMap { currentRow => + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + val buildIter = hashedRelation.get(rowKey) + new RowIterator { + private var found = false + override def advanceNext(): Boolean = { + while (buildIter != null && buildIter.hasNext) { + val nextBuildRow = buildIter.next() + if (boundCondition(joinedRow.withRight(nextBuildRow))) { + found = true + return true + } + } + if (!found) { + joinedRow.withRight(nullRow) found = true return true } + false } - if (!found) { - joinedRow.withRight(nullRow) - found = true - return true - } - false - } - override def getRow: InternalRow = joinedRow - }.toScala + override def getRow: InternalRow = joinedRow + }.toScala + } } } @@ -148,12 +219,22 @@ trait HashJoin extends BaseJoinExec { hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinKeys = streamSideKeyGenerator() val joinedRow = new JoinedRow - streamIter.filter { current => - val key = joinKeys(current) - lazy val buildIter = hashedRelation.get(key) - !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { - (row: InternalRow) => boundCondition(joinedRow(current, row)) - }) + + if (hashedRelation.keyIsUnique) { + streamIter.filter { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + !key.anyNull && matched != null && + (condition.isEmpty || boundCondition(joinedRow(current, matched))) + } + } else { + streamIter.filter { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + } } } @@ -163,14 +244,26 @@ trait HashJoin extends BaseJoinExec { val joinKeys = streamSideKeyGenerator() val result = new GenericInternalRow(Array[Any](null)) val joinedRow = new JoinedRow - streamIter.map { current => - val key = joinKeys(current) - lazy val buildIter = hashedRelation.get(key) - val exists = !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { - (row: InternalRow) => boundCondition(joinedRow(current, row)) - }) - result.setBoolean(0, exists) - joinedRow(current, result) + + if (hashedRelation.keyIsUnique) { + streamIter.map { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + val exists = !key.anyNull && matched != null && + (condition.isEmpty || boundCondition(joinedRow(current, matched))) + result.setBoolean(0, exists) + joinedRow(current, result) + } + } else { + streamIter.map { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + val exists = !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + result.setBoolean(0, exists) + joinedRow(current, result) + } } } @@ -179,12 +272,22 @@ trait HashJoin extends BaseJoinExec { hashedRelation: HashedRelation): Iterator[InternalRow] = { val joinKeys = streamSideKeyGenerator() val joinedRow = new JoinedRow - streamIter.filter { current => - val key = joinKeys(current) - lazy val buildIter = hashedRelation.get(key) - key.anyNull || buildIter == null || (condition.isDefined && !buildIter.exists { - row => boundCondition(joinedRow(current, row)) - }) + + if (hashedRelation.keyIsUnique) { + streamIter.filter { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + key.anyNull || matched == null || + (condition.isDefined && !boundCondition(joinedRow(current, matched))) + } + } else { + streamIter.filter { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + key.anyNull || buildIter == null || (condition.isDefined && !buildIter.exists { + row => boundCondition(joinedRow(current, row)) + }) + } } } @@ -202,11 +305,11 @@ trait HashJoin extends BaseJoinExec { semiJoin(streamedIter, hashed) case LeftAnti => antiJoin(streamedIter, hashed) - case j: ExistenceJoin => + case _: ExistenceJoin => existenceJoin(streamedIter, hashed) case x => throw new IllegalArgumentException( - s"BroadcastHashJoin should not take $x as the JoinType") + s"HashJoin should not take $x as the JoinType") } val resultProj = createResultProjection @@ -215,6 +318,409 @@ trait HashJoin extends BaseJoinExec { resultProj(r) } } + + override def doProduce(ctx: CodegenContext): String = { + streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this) + } + + override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { + joinType match { + case _: InnerLike => codegenInner(ctx, input) + case LeftOuter | RightOuter => codegenOuter(ctx, input) + case LeftSemi => codegenSemi(ctx, input) + case LeftAnti => codegenAnti(ctx, input) + case _: ExistenceJoin => codegenExistence(ctx, input) + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType") + } + } + + /** + * Returns the code for generating join key for stream side, and expression of whether the key + * has any null in it or not. + */ + protected def genStreamSideJoinKey( + ctx: CodegenContext, + input: Seq[ExprCode]): (ExprCode, String) = { + ctx.currentVars = input + if (streamedBoundKeys.length == 1 && streamedBoundKeys.head.dataType == LongType) { + // generate the join key as Long + val ev = streamedBoundKeys.head.genCode(ctx) + (ev, ev.isNull) + } else { + // generate the join key as UnsafeRow + val ev = GenerateUnsafeProjection.createCode(ctx, streamedBoundKeys) + (ev, s"${ev.value}.anyNull()") + } + } + + /** + * Generates the code for variable of build side. + */ + private def genBuildSideVars(ctx: CodegenContext, matched: String): Seq[ExprCode] = { + ctx.currentVars = null + ctx.INPUT_ROW = matched + buildPlan.output.zipWithIndex.map { case (a, i) => + val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx) + if (joinType.isInstanceOf[InnerLike]) { + ev + } else { + // the variables are needed even there is no matched rows + val isNull = ctx.freshName("isNull") + val value = ctx.freshName("value") + val javaType = CodeGenerator.javaType(a.dataType) + val code = code""" + |boolean $isNull = true; + |$javaType $value = ${CodeGenerator.defaultValue(a.dataType)}; + |if ($matched != null) { + | ${ev.code} + | $isNull = ${ev.isNull}; + | $value = ${ev.value}; + |} + """.stripMargin + ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType)) + } + } + } + + /** + * Generate the (non-equi) condition used to filter joined rows. This is used in Inner, Left Semi + * and Left Anti joins. + */ + protected def getJoinCondition( + ctx: CodegenContext, + input: Seq[ExprCode]): (String, String, Seq[ExprCode]) = { + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + // filter the output via condition + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + val skipRow = s"${ev.isNull} || !${ev.value}" + s""" + |$eval + |${ev.code} + |if (!($skipRow)) + """.stripMargin + } else { + "" + } + (matched, checkCondition, buildVars) + } + + /** + * Generates the code for Inner join. + */ + protected def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + + val resultVars = buildSide match { + case BuildLeft => buildVars ++ input + case BuildRight => input ++ buildVars + } + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |if ($matched != null) { + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? + | null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |if ($matches != null) { + | while ($matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for left or right outer join. + */ + protected def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val numOutput = metricTerm(ctx, "numOutputRows") + + // filter the output via condition + val conditionPassed = ctx.freshName("conditionPassed") + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |boolean $conditionPassed = true; + |${eval.trim} + |if ($matched != null) { + | ${ev.code} + | $conditionPassed = !${ev.isNull} && ${ev.value}; + |} + """.stripMargin + } else { + s"final boolean $conditionPassed = true;" + } + + val resultVars = buildSide match { + case BuildLeft => buildVars ++ input + case BuildRight => input ++ buildVars + } + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |${checkCondition.trim} + |if (!$conditionPassed) { + | $matched = null; + | // reset the variables those are already evaluated. + | ${buildVars.filter(_.code.isEmpty).map(v => s"${v.isNull} = true;").mkString("\n")} + |} + |$numOutput.add(1); + |${consume(ctx, resultVars)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $found = false; + |// the last iteration of this loop is to emit an empty row if there is no matched rows. + |while ($matches != null && $matches.hasNext() || !$found) { + | UnsafeRow $matched = $matches != null && $matches.hasNext() ? + | (UnsafeRow) $matches.next() : null; + | ${checkCondition.trim} + | if ($conditionPassed) { + | $found = true; + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for left semi join. + */ + protected def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, _) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |if ($matched != null) { + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, input)} + | } + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |if ($matches != null) { + | boolean $found = false; + | while (!$found && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $found = true; + | } + | } + | if ($found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for anti join. + */ + protected def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, _) = getJoinCondition(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + + if (keyIsUnique) { + val found = ctx.freshName("found") + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |// Check if the key has nulls. + |if (!($anyNull)) { + | // Check if the HashedRelation exists. + | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + | if ($matched != null) { + | // Evaluate the condition. + | $checkCondition { + | $found = true; + | } + | } + |} + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |// Check if the key has nulls. + |if (!($anyNull)) { + | // Check if the HashedRelation exists. + | $iteratorCls $matches = ($iteratorCls)$relationTerm.get(${keyEv.value}); + | if ($matches != null) { + | // Evaluate the condition. + | while (!$found && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $found = true; + | } + | } + | } + |} + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } + } + + /** + * Generates the code for existence join. + */ + protected def codegenExistence(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (relationTerm, keyIsUnique) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + val existsVar = ctx.freshName("exists") + + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + // filter the output via condition + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |$eval + |${ev.code} + |$existsVar = !${ev.isNull} && ${ev.value}; + """.stripMargin + } else { + s"$existsVar = true;" + } + + val resultVar = input ++ Seq(ExprCode.forNonNullValue( + JavaCode.variable(existsVar, BooleanType))) + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |boolean $existsVar = false; + |if ($matched != null) { + | $checkCondition + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $existsVar = false; + |if ($matches != null) { + | while (!$existsVar && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition + | } + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } + } + + /** + * Returns a tuple of variable name for HashedRelation, + * and a boolean to indicate whether keys of HashedRelation + * known to be unique in code-gen time. + */ + protected def prepareRelation(ctx: CodegenContext): (String, Boolean) } object HashJoin { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 13180d6b20902..f2835c2fa6626 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -96,7 +96,8 @@ private[execution] object HashedRelation { input: Iterator[InternalRow], key: Seq[Expression], sizeEstimate: Int = 64, - taskMemoryManager: TaskMemoryManager = null): HashedRelation = { + taskMemoryManager: TaskMemoryManager = null, + isNullAware: Boolean = false): HashedRelation = { val mm = Option(taskMemoryManager).getOrElse { new TaskMemoryManager( new UnifiedMemoryManager( @@ -107,10 +108,12 @@ private[execution] object HashedRelation { 0) } - if (key.length == 1 && key.head.dataType == LongType) { - LongHashedRelation(input, key, sizeEstimate, mm) + if (isNullAware && !input.hasNext) { + EmptyHashedRelation + } else if (key.length == 1 && key.head.dataType == LongType) { + LongHashedRelation(input, key, sizeEstimate, mm, isNullAware) } else { - UnsafeHashedRelation(input, key, sizeEstimate, mm) + UnsafeHashedRelation(input, key, sizeEstimate, mm, isNullAware) } } } @@ -310,7 +313,8 @@ private[joins] object UnsafeHashedRelation { input: Iterator[InternalRow], key: Seq[Expression], sizeEstimate: Int, - taskMemoryManager: TaskMemoryManager): HashedRelation = { + taskMemoryManager: TaskMemoryManager, + isNullAware: Boolean = false): HashedRelation = { val pageSizeBytes = Option(SparkEnv.get).map(_.memoryManager.pageSizeBytes) .getOrElse(new SparkConf().get(BUFFER_PAGESIZE).getOrElse(16L * 1024 * 1024)) @@ -338,6 +342,8 @@ private[joins] object UnsafeHashedRelation { throw new SparkOutOfMemoryError("There is not enough memory to build hash map") // scalastyle:on throwerror } + } else if (isNullAware) { + return EmptyHashedRelationWithAllNullKeys } } @@ -889,7 +895,8 @@ private[joins] object LongHashedRelation { input: Iterator[InternalRow], key: Seq[Expression], sizeEstimate: Int, - taskMemoryManager: TaskMemoryManager): LongHashedRelation = { + taskMemoryManager: TaskMemoryManager, + isNullAware: Boolean = false): HashedRelation = { val map = new LongToUnsafeRowMap(taskMemoryManager, sizeEstimate) val keyGenerator = UnsafeProjection.create(key) @@ -903,6 +910,8 @@ private[joins] object LongHashedRelation { if (!rowKey.isNullAt(0)) { val key = rowKey.getLong(0) map.append(key, unsafeRow) + } else if (isNullAware) { + return EmptyHashedRelationWithAllNullKeys } } map.optimize() @@ -910,8 +919,52 @@ private[joins] object LongHashedRelation { } } +/** + * Common trait with dummy implementation for NAAJ special HashedRelation + * EmptyHashedRelation + * EmptyHashedRelationWithAllNullKeys + */ +trait NullAwareHashedRelation extends HashedRelation with Externalizable { + override def get(key: InternalRow): Iterator[InternalRow] = { + throw new UnsupportedOperationException + } + + override def getValue(key: InternalRow): InternalRow = { + throw new UnsupportedOperationException + } + + override def keyIsUnique: Boolean = true + + override def keys(): Iterator[InternalRow] = { + throw new UnsupportedOperationException + } + + override def close(): Unit = {} + + override def writeExternal(out: ObjectOutput): Unit = {} + + override def readExternal(in: ObjectInput): Unit = {} + + override def estimatedSize: Long = 0 +} + +/** + * A special HashedRelation indicates it built from a empty input:Iterator[InternalRow]. + */ +object EmptyHashedRelation extends NullAwareHashedRelation { + override def asReadOnlyCopy(): EmptyHashedRelation.type = this +} + +/** + * A special HashedRelation indicates it built from a non-empty input:Iterator[InternalRow], + * which contains all null columns key. + */ +object EmptyHashedRelationWithAllNullKeys extends NullAwareHashedRelation { + override def asReadOnlyCopy(): EmptyHashedRelationWithAllNullKeys.type = this +} + /** The HashedRelationBroadcastMode requires that rows are broadcasted as a HashedRelation. */ -case class HashedRelationBroadcastMode(key: Seq[Expression]) +case class HashedRelationBroadcastMode(key: Seq[Expression], isNullAware: Boolean = false) extends BroadcastMode { override def transform(rows: Array[InternalRow]): HashedRelation = { @@ -923,9 +976,9 @@ case class HashedRelationBroadcastMode(key: Seq[Expression]) sizeHint: Option[Long]): HashedRelation = { sizeHint match { case Some(numRows) => - HashedRelation(rows, canonicalized.key, numRows.toInt) + HashedRelation(rows, canonicalized.key, numRows.toInt, isNullAware = isNullAware) case None => - HashedRelation(rows, canonicalized.key) + HashedRelation(rows, canonicalized.key, isNullAware = isNullAware) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 2b7cd65e7d96f..9f811cddef6a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -23,6 +23,7 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.optimizer.BuildSide import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -40,22 +41,25 @@ case class ShuffledHashJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan) - extends HashJoin { + extends HashJoin with ShuffledJoin { override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) - override def requiredChildDistribution: Seq[Distribution] = - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + override def outputPartitioning: Partitioning = super[ShuffledJoin].outputPartitioning - private def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = { + /** + * This is called by generated Java class, should be public. + */ + def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = { val buildDataSize = longMetric("buildDataSize") val buildTime = longMetric("buildTime") val start = System.nanoTime() val context = TaskContext.get() - val relation = HashedRelation(iter, buildKeys, taskMemoryManager = context.taskMemoryManager()) + val relation = HashedRelation( + iter, buildBoundKeys, taskMemoryManager = context.taskMemoryManager()) buildTime += NANOSECONDS.toMillis(System.nanoTime() - start) buildDataSize += relation.estimatedSize // This relation is usually used until the end of task. @@ -70,4 +74,20 @@ case class ShuffledHashJoinExec( join(streamIter, hashed, numOutputRows) } } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + streamedPlan.execute() :: buildPlan.execute() :: Nil + } + + override def needCopyResult: Boolean = true + + protected override def prepareRelation(ctx: CodegenContext): (String, Boolean) = { + val thisPlan = ctx.addReferenceObj("plan", this) + val clsName = classOf[HashedRelation].getName + + // Inline mutable state since not many join operations in a task + val relationTerm = ctx.addMutableState(clsName, "relation", + v => s"$v = $thisPlan.buildHashedRelation(inputs[1]);", forceInline = true) + (relationTerm, false) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala new file mode 100644 index 0000000000000..7035ddc35be9c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.sql.catalyst.plans.{FullOuter, InnerLike, LeftExistence, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution, Partitioning, PartitioningCollection, UnknownPartitioning} + +/** + * Holds common logic for join operators by shuffling two child relations + * using the join keys. + */ +trait ShuffledJoin extends BaseJoinExec { + override def requiredChildDistribution: Seq[Distribution] = { + HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + } + + override def outputPartitioning: Partitioning = joinType match { + case _: InnerLike => + PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning)) + case LeftOuter => left.outputPartitioning + case RightOuter => right.outputPartitioning + case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) + case LeftExistence(_) => left.outputPartitioning + case x => + throw new IllegalArgumentException( + s"ShuffledJoin should not take $x as the JoinType") + } +} 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 2c57956de5bca..b9f6684447dd8 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 @@ -41,7 +41,7 @@ case class SortMergeJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan, - isSkewJoin: Boolean = false) extends BaseJoinExec with CodegenSupport { + isSkewJoin: Boolean = false) extends ShuffledJoin with CodegenSupport { override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -72,26 +72,13 @@ case class SortMergeJoinExec( } } - override def outputPartitioning: Partitioning = joinType match { - case _: InnerLike => - PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning)) - // For left and right outer joins, the output is partitioned by the streamed input's join keys. - case LeftOuter => left.outputPartitioning - case RightOuter => right.outputPartitioning - case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) - case LeftExistence(_) => left.outputPartitioning - case x => - throw new IllegalArgumentException( - s"${getClass.getSimpleName} should not take $x as the JoinType") - } - override def requiredChildDistribution: Seq[Distribution] = { if (isSkewJoin) { // We re-arrange the shuffle partitions to deal with skew join, and the new children // partitioning doesn't satisfy `HashClusteredDistribution`. UnspecifiedDistribution :: UnspecifiedDistribution :: Nil } else { - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + super.requiredChildDistribution } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 4b2d4195ee906..c08db132c946f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -567,7 +567,14 @@ case class FlatMapGroupsInRWithArrowExec( // binary in a batch due to the limitation of R API. See also ARROW-4512. val columnarBatchIter = runner.compute(groupedByRKey, -1) val outputProject = UnsafeProjection.create(output, output) - columnarBatchIter.flatMap(_.rowIterator().asScala).map(outputProject) + val outputTypes = StructType.fromAttributes(output).map(_.dataType) + + columnarBatchIter.flatMap { batch => + val actualDataTypes = (0 until batch.numCols()).map(i => batch.column(i).dataType()) + assert(outputTypes == actualDataTypes, "Invalid schema from gapply(): " + + s"expected ${outputTypes.mkString(", ")}, got ${actualDataTypes.mkString(", ")}") + batch.rowIterator().asScala + }.map(outputProject) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 7773ac71c4954..bfa60cf7dfd78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, HashPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{LeafExecNode, LocalLimitExec, QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.util.Utils @@ -118,7 +118,7 @@ class IncrementalExecution( case s: StatefulOperator => statefulOpFound = true - case e: ShuffleExchangeExec => + case e: ShuffleExchangeLike => // Don't search recursively any further as any child stateful operator as we // are only looking for stateful subplans that this plan has narrow dependencies on. 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 f9e6e2f7b359c..653fe5bf7c9b3 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 @@ -461,7 +461,7 @@ object functions { * @since 2.0.0 */ def first(e: Column, ignoreNulls: Boolean): Column = withAggregateFunction { - new First(e.expr, Literal(ignoreNulls)) + First(e.expr, ignoreNulls) } /** @@ -586,7 +586,7 @@ object functions { * @since 2.0.0 */ def last(e: Column, ignoreNulls: Boolean): Column = withAggregateFunction { - new Last(e.expr, Literal(ignoreNulls)) + new Last(e.expr, ignoreNulls) } /** @@ -2478,6 +2478,8 @@ object functions { /** * Extract a specific group matched by a Java regex, from the specified string column. * If the regex did not match, or the specified group did not match, an empty string is returned. + * if the specified group index exceeds the group count of regex, an IllegalArgumentException + * will be thrown. * * @group string_funcs * @since 1.5.0 @@ -2486,6 +2488,19 @@ object functions { RegExpExtract(e.expr, lit(exp).expr, lit(groupIdx).expr) } + /** + * Extract all specific groups matched by a Java regex, from the specified string column. + * If the regex did not match, or the specified group did not match, return an empty array. + * if the specified group index exceeds the group count of regex, an IllegalArgumentException + * will be thrown. + * + * @group string_funcs + * @since 3.1.0 + */ + def regexp_extract_all(e: Column, exp: String, groupIdx: Int): Column = withExpr { + RegExpExtractAll(e.expr, lit(exp).expr, lit(groupIdx).expr) + } + /** * Replace all substrings of the specified string value that match regexp with rep. * @@ -3402,8 +3417,22 @@ object functions { * @group collection_funcs * @since 2.4.0 */ - def slice(x: Column, start: Int, length: Int): Column = withExpr { - Slice(x.expr, Literal(start), Literal(length)) + def slice(x: Column, start: Int, length: Int): Column = + slice(x, lit(start), lit(length)) + + /** + * Returns an array containing all the elements in `x` from index `start` (or starting from the + * end if `start` is negative) with the specified `length`. + * + * @param x the array column to be sliced + * @param start the starting index + * @param length the length of the slice + * + * @group collection_funcs + * @since 3.1.0 + */ + def slice(x: Column, start: Column, length: Column): Column = withExpr { + Slice(x.expr, start.expr, length.expr) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 4ae12f8716752..83a7a557305e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} +import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlan, SparkPlanner, SparkSqlParser} import org.apache.spark.sql.execution.aggregate.ResolveEncodersInScalaAgg import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.command.CommandCheck @@ -286,6 +286,10 @@ abstract class BaseSessionStateBuilder( extensions.buildColumnarRules(session) } + protected def queryStagePrepRules: Seq[Rule[SparkPlan]] = { + extensions.buildQueryStagePrepRules(session) + } + /** * Create a query execution object. */ @@ -337,7 +341,8 @@ abstract class BaseSessionStateBuilder( () => resourceLoader, createQueryExecution, createClone, - columnarRules) + columnarRules, + queryStagePrepRules) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index abd1250628539..cd425b04ef311 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager @@ -73,7 +74,8 @@ private[sql] class SessionState( resourceLoaderBuilder: () => SessionResourceLoader, createQueryExecution: LogicalPlan => QueryExecution, createClone: (SparkSession, SessionState) => SessionState, - val columnarRules: Seq[ColumnarRule]) { + val columnarRules: Seq[ColumnarRule], + val queryStagePrepRules: Seq[Rule[SparkPlan]]) { // The following fields are lazy to avoid creating the Hive client when creating SessionState. lazy val catalog: SessionCatalog = catalogBuilder() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala index ab574df4557a5..430ca9edab799 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala @@ -51,4 +51,11 @@ private object DB2Dialect extends JdbcDialect { } override def isCascadingTruncateTable(): Option[Boolean] = Some(false) + + // scalastyle:off line.size.limit + // See https://www.ibm.com/support/knowledgecenter/en/SSEPGG_11.5.0/com.ibm.db2.luw.sql.ref.doc/doc/r0000980.html + // scalastyle:on line.size.limit + override def renameTable(oldTable: String, newTable: String): String = { + s"RENAME TABLE $oldTable TO $newTable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala index d528d5a9fef5a..9ca8879be31e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala @@ -45,4 +45,9 @@ private object DerbyDialect extends JdbcDialect { } override def isCascadingTruncateTable(): Option[Boolean] = Some(false) + + // See https://db.apache.org/derby/docs/10.5/ref/rrefsqljrenametablestatement.html + override def renameTable(oldTable: String, newTable: String): String = { + s"RENAME TABLE $oldTable TO $newTable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index a0c6d20f36451..cea5a20917532 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -17,11 +17,16 @@ package org.apache.spark.sql.jdbc -import java.sql.{Connection, Date, Timestamp} +import java.sql.{Connection, Date, SQLFeatureNotSupportedException, Timestamp} + +import scala.collection.mutable.ArrayBuilder import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.sql.connector.catalog.TableChange +import org.apache.spark.sql.connector.catalog.TableChange._ +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils import org.apache.spark.sql.types._ /** @@ -180,6 +185,55 @@ abstract class JdbcDialect extends Serializable { * None: The behavior of TRUNCATE TABLE is unknown (default). */ def isCascadingTruncateTable(): Option[Boolean] = None + + /** + * Rename an existing table. + * + * @param oldTable The existing table. + * @param newTable New name of the table. + * @return The SQL statement to use for renaming the table. + */ + def renameTable(oldTable: String, newTable: String): String = { + s"ALTER TABLE $oldTable RENAME TO $newTable" + } + + /** + * Alter an existing table. + * TODO (SPARK-32523): Override this method in the dialects that have different syntax. + * + * @param tableName The name of the table to be altered. + * @param changes Changes to apply to the table. + * @return The SQL statements to use for altering the table. + */ + def alterTable(tableName: String, changes: Seq[TableChange]): Array[String] = { + val updateClause = ArrayBuilder.make[String] + for (change <- changes) { + change match { + case add: AddColumn if add.fieldNames.length == 1 => + val dataType = JdbcUtils.getJdbcType(add.dataType(), this).databaseTypeDefinition + val name = add.fieldNames + updateClause += s"ALTER TABLE $tableName ADD COLUMN ${name(0)} $dataType" + case rename: RenameColumn if rename.fieldNames.length == 1 => + val name = rename.fieldNames + updateClause += s"ALTER TABLE $tableName RENAME COLUMN ${name(0)} TO ${rename.newName}" + case delete: DeleteColumn if delete.fieldNames.length == 1 => + val name = delete.fieldNames + updateClause += s"ALTER TABLE $tableName DROP COLUMN ${name(0)}" + case updateColumnType: UpdateColumnType if updateColumnType.fieldNames.length == 1 => + val name = updateColumnType.fieldNames + val dataType = JdbcUtils.getJdbcType(updateColumnType.newDataType(), this) + .databaseTypeDefinition + updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} $dataType" + case updateNull: UpdateColumnNullability if updateNull.fieldNames.length == 1 => + val name = updateNull.fieldNames + val nullable = if (updateNull.nullable()) "NULL" else "NOT NULL" + updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} SET $nullable" + case _ => + throw new SQLFeatureNotSupportedException(s"Unsupported TableChange $change") + } + } + updateClause.result() + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala index 72284b5996201..1c6e8c359aa15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala @@ -57,4 +57,11 @@ private object MsSqlServerDialect extends JdbcDialect { } override def isCascadingTruncateTable(): Option[Boolean] = Some(false) + + // scalastyle:off line.size.limit + // See https://docs.microsoft.com/en-us/sql/relational-databases/system-stored-procedures/sp-rename-transact-sql?view=sql-server-ver15 + // scalastyle:on line.size.limit + override def renameTable(oldTable: String, newTable: String): String = { + s"EXEC sp_rename $oldTable, $newTable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala index 552d7a484f3fd..58fe62cb6e088 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala @@ -50,4 +50,9 @@ private case object TeradataDialect extends JdbcDialect { cascade: Option[Boolean] = isCascadingTruncateTable): String = { s"DELETE FROM $table ALL" } + + // See https://docs.teradata.com/reader/scPHvjfglIlB8F70YliLAw/wysTNUMsP~0aGzksLCl1kg + override def renameTable(oldTable: String, newTable: String): String = { + s"RENAME TABLE $oldTable TO $newTable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 1d7e4d3059c68..2b0db4381c6e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -308,6 +308,14 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * It does not change the behavior of partition discovery.
          • *
          • `recursiveFileLookup`: recursively scan a directory for files. Using this option * disables partition discovery
          • + *
          • `allowNonNumericNumbers` (default `true`): allows JSON parser to recognize set of + * "Not-a-Number" (NaN) tokens as legal floating number values: + *
              + *
            • `+INF` for positive infinity, as well as alias of `+Infinity` and `Infinity`. + *
            • `-INF` for negative infinity, alias `-Infinity`. + *
            • `NaN` for other not-a-numbers, like result of division by zero. + *
            + *
          • *
          * * @since 2.0.0 diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 564e76737ecde..6cb7c40e3332b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -37,6 +37,8 @@ import org.junit.Before; import org.junit.Test; +import java.nio.ByteOrder; + /** * Test the RecordBinaryComparator, which compares two UnsafeRows by their binary form. */ @@ -261,40 +263,74 @@ public void testBinaryComparatorForNullColumns() throws Exception { public void testBinaryComparatorWhenSubtractionIsDivisibleByMaxIntValue() throws Exception { int numFields = 1; + // Place the following bytes (hex) into UnsafeRows for the comparison: + // + // index | 00 01 02 03 04 05 06 07 + // ------+------------------------ + // row1 | 00 00 00 00 00 00 00 0b + // row2 | 00 00 00 00 80 00 00 0a + // + // The byte layout needs to be identical on all platforms regardless of + // of endianness. To achieve this the bytes in each value are reversed + // on little-endian platforms. + long row1Data = 11L; + long row2Data = 11L + Integer.MAX_VALUE; + if (ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { + row1Data = Long.reverseBytes(row1Data); + row2Data = Long.reverseBytes(row2Data); + } + UnsafeRow row1 = new UnsafeRow(numFields); byte[] data1 = new byte[100]; row1.pointTo(data1, computeSizeInBytes(numFields * 8)); - row1.setLong(0, 11); + row1.setLong(0, row1Data); UnsafeRow row2 = new UnsafeRow(numFields); byte[] data2 = new byte[100]; row2.pointTo(data2, computeSizeInBytes(numFields * 8)); - row2.setLong(0, 11L + Integer.MAX_VALUE); + row2.setLong(0, row2Data); insertRow(row1); insertRow(row2); - Assert.assertTrue(compare(0, 1) > 0); + Assert.assertTrue(compare(0, 1) < 0); } @Test public void testBinaryComparatorWhenSubtractionCanOverflowLongValue() throws Exception { int numFields = 1; + // Place the following bytes (hex) into UnsafeRows for the comparison: + // + // index | 00 01 02 03 04 05 06 07 + // ------+------------------------ + // row1 | 80 00 00 00 00 00 00 00 + // row2 | 00 00 00 00 00 00 00 01 + // + // The byte layout needs to be identical on all platforms regardless of + // of endianness. To achieve this the bytes in each value are reversed + // on little-endian platforms. + long row1Data = Long.MIN_VALUE; + long row2Data = 1L; + if (ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { + row1Data = Long.reverseBytes(row1Data); + row2Data = Long.reverseBytes(row2Data); + } + UnsafeRow row1 = new UnsafeRow(numFields); byte[] data1 = new byte[100]; row1.pointTo(data1, computeSizeInBytes(numFields * 8)); - row1.setLong(0, Long.MIN_VALUE); + row1.setLong(0, row1Data); UnsafeRow row2 = new UnsafeRow(numFields); byte[] data2 = new byte[100]; row2.pointTo(data2, computeSizeInBytes(numFields * 8)); - row2.setLong(0, 1); + row2.setLong(0, row2Data); insertRow(row1); insertRow(row2); - Assert.assertTrue(compare(0, 1) < 0); + Assert.assertTrue(compare(0, 1) > 0); } @Test diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 8898a11ec08fb..a212d8ce40642 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -214,6 +214,7 @@ | org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | struct | | org.apache.spark.sql.catalyst.expressions.Rank | rank | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.RegExpExtract | regexp_extract | SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1) | struct | +| org.apache.spark.sql.catalyst.expressions.RegExpExtractAll | regexp_extract_all | SELECT regexp_extract_all('100-200, 300-400', '(\\d+)-(\\d+)', 1) | struct> | | org.apache.spark.sql.catalyst.expressions.RegExpReplace | regexp_replace | SELECT regexp_replace('100-200', '(\\d+)', 'num') | struct | | org.apache.spark.sql.catalyst.expressions.Remainder | % | SELECT 2 % 1.8 | struct<(CAST(CAST(2 AS DECIMAL(1,0)) AS DECIMAL(2,1)) % CAST(1.8 AS DECIMAL(2,1))):decimal(2,1)> | | org.apache.spark.sql.catalyst.expressions.Remainder | mod | SELECT 2 % 1.8 | struct<(CAST(CAST(2 AS DECIMAL(1,0)) AS DECIMAL(2,1)) % CAST(1.8 AS DECIMAL(2,1))):decimal(2,1)> | @@ -291,7 +292,7 @@ | org.apache.spark.sql.catalyst.expressions.Uuid | uuid | SELECT uuid() | struct | | org.apache.spark.sql.catalyst.expressions.WeekDay | weekday | SELECT weekday('2009-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.WeekOfYear | weekofyear | SELECT weekofyear('2008-02-20') | struct | -| org.apache.spark.sql.catalyst.expressions.WidthBucket | width_bucket | SELECT width_bucket(5.3, 0.2, 10.6, 5) | struct | +| org.apache.spark.sql.catalyst.expressions.WidthBucket | width_bucket | SELECT width_bucket(5.3, 0.2, 10.6, 5) | struct | | org.apache.spark.sql.catalyst.expressions.XxHash64 | xxhash64 | SELECT xxhash64('Spark', array(123), 2) | struct | | org.apache.spark.sql.catalyst.expressions.Year | year | SELECT year('2016-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | struct>> | @@ -315,12 +316,12 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAgg | count_min_sketch | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.aggregate.CovPopulation | covar_pop | SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.CovSample | covar_samp | SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus | approx_count_distinct | SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis | kurtosis | SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Max | max | SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy | max_by | SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Min | min | SELECT min(col) FROM VALUES (10), (-1), (20) AS tab(col) | struct | diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql index 9072aa107f252..11bba00e91abf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting-invalid.sql @@ -3,8 +3,8 @@ -- separating this from datetime-formatting.sql, because the text form -- for patterns with 5 letters in SimpleDateFormat varies from different JDKs select date_format('2018-11-17 13:33:33.333', 'GGGGG'); --- pattern letter count can not be greater than 10 -select date_format('2018-11-17 13:33:33.333', 'yyyyyyyyyyy'); +-- pattern letter count can not be greater than 6 +select date_format('2018-11-17 13:33:33.333', 'yyyyyyy'); -- q/L in JDK 8 will fail when the count is more than 2 select date_format('2018-11-17 13:33:33.333', 'qqqqq'); select date_format('2018-11-17 13:33:33.333', 'QQQQQ'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql index 3b23a7785f6cd..2d70326f4f3c0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-formatting.sql @@ -11,7 +11,7 @@ create temporary view v as select col from values select col, date_format(col, 'G GG GGG GGGG') from v; -select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy yyyyyyy yyyyyyyy yyyyyyyyy yyyyyyyyyy') from v; +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v; select col, date_format(col, 'q qq') from v; diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql index 0b313e5a0b9f4..a1c02eaa3b0a0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing-invalid.sql @@ -1,5 +1,14 @@ --- TESTS FOR DATETIME PARSING FUNCTIONS WITH INVALID VALUES --- +-- parsing invalid value with pattern 'y' +select to_timestamp('294248', 'y'); -- out of year value range [0, 294247] +select to_timestamp('1', 'yy'); -- the number of digits must be 2 for 'yy'. +select to_timestamp('-12', 'yy'); -- out of year value range [0, 99] for reduced two digit form +select to_timestamp('123', 'yy'); -- the number of digits must be 2 for 'yy'. +select to_timestamp('1', 'yyy'); -- the number of digits must be in [3, 6] for 'yyy' + +select to_timestamp('1234567', 'yyyyyyy'); -- the length of 'y' pattern must be less than 7 + -- parsing invalid values with pattern 'D' select to_timestamp('366', 'D'); select to_timestamp('9', 'DD'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql index 74866d9c6ffa1..e058bd675c375 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime-parsing.sql @@ -1,5 +1,33 @@ --- TESTS FOR DATETIME PARSING FUNCTIONS --- +-- parsing with pattern 'y'. +-- the range of valid year is [-290307, 294247], +-- but particularly, some thrift client use java.sql.Timestamp to parse timestamp, which allows +-- only positive year values less or equal than 9999. So the cases bellow only use [1, 9999] to pass +-- ThriftServerQueryTestSuite +select to_timestamp('1', 'y'); +select to_timestamp('009999', 'y'); + +-- reduced two digit form is used, the range of valid year is 20-[01, 99] +select to_timestamp('00', 'yy'); +select to_timestamp('99', 'yy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be in [3, 6] for 'yyy' +select to_timestamp('001', 'yyy'); +select to_timestamp('009999', 'yyy'); + +-- the range of valid year is [-9999, 9999], the number of digits must be 4 for 'yyyy'. +select to_timestamp('0001', 'yyyy'); +select to_timestamp('9999', 'yyyy'); + +-- the range of valid year is [-99999, 99999], the number of digits must be 5 for 'yyyyy'. +select to_timestamp('00001', 'yyyyy'); +select to_timestamp('09999', 'yyyyy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be 6 for 'yyyyyy'. +select to_timestamp('000001', 'yyyyyy'); +select to_timestamp('009999', 'yyyyyy'); + -- parsing with pattern 'D' select to_timestamp('9', 'D'); select to_timestamp('300', 'D'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql index beb5b9e5fe516..24d303621faea 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql @@ -1,5 +1,8 @@ -- Test filter clause for aggregate expression. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) @@ -33,8 +36,13 @@ SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp; SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData; +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp; -- Aggregate with filter and non-empty GroupBy expressions. SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a; @@ -44,8 +52,11 @@ SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id; SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id; SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id; +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b; -- Aggregate with filter and grouped by literals. SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1; @@ -58,13 +69,24 @@ select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id; --- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id; +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id; -- Aggregate with filter and grouped by literals (hash aggregate), here the input table is filtered using WHERE. SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1; @@ -78,9 +100,8 @@ SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1; SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1; -- Aggregate with filter, foldable input and multiple distinct groups. --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) --- FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; -- Check analysis exceptions SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql index 746b677234832..657ea59ec8f11 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql @@ -241,10 +241,9 @@ select sum(1/ten) filter (where ten > 0) from tenk1; -- select ten, sum(distinct four) filter (where four::text ~ '123') from onek a -- group by ten; --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select ten, sum(distinct four) filter (where four > 10) from onek a --- group by ten --- having exists (select 1 from onek b where sum(distinct a.four) = b.four); +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four); -- [SPARK-28682] ANSI SQL: Collation Support -- select max(foo COLLATE "C") filter (where (bar collate "POSIX") > '0') diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql index fc54d179f742c..45617c53166aa 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql @@ -336,9 +336,8 @@ order by 2,1; -- order by 2,1; -- FILTER queries --- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT --- select ten, sum(distinct four) filter (where string(four) like '123') from onek a --- group by rollup(ten); +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten); -- More rescan tests -- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) diff --git a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql index c0827a3cba39b..7128dee0a00d7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql @@ -3,7 +3,31 @@ SELECT regexp_extract('1a 2b 14m', '\\d+'); SELECT regexp_extract('1a 2b 14m', '\\d+', 0); SELECT regexp_extract('1a 2b 14m', '\\d+', 1); SELECT regexp_extract('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract('a b m', '(\\d+)?', 1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)'); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1); SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1); + +-- regexp_extract_all +SELECT regexp_extract_all('1a 2b 14m', '\\d+'); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql b/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql index 2861b2b43a113..0ba936dcad741 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql @@ -17,8 +17,8 @@ SHOW TBLPROPERTIES view("p3"); DROP VIEW view; --- create a temporary view with properties -CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1; +-- create a temporary view +CREATE TEMPORARY VIEW tv AS SELECT 1 AS c1; -- Properties for a temporary view should be empty SHOW TBLPROPERTIES tv; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql index f4ffc20086386..5669423148f80 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-basic.sql @@ -1,3 +1,6 @@ +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view tab_a as select * from values (1, 1) as tab_a(a1, b1); create temporary view tab_b as select * from values (1, 1) as tab_b(a2, b2); create temporary view struct_tab as select struct(col1 as a, col2 as b) as record from diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql index 8f98ae1155062..750cc42b8641c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql @@ -1,6 +1,9 @@ -- A test suite for IN HAVING in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql index 200a71ebbb622..2353560137d21 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -13,6 +13,9 @@ --CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY --CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN +--CONFIG_DIM3 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM3 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql index 0a16f118f0455..53fc2b8be7501 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -1,6 +1,9 @@ -- A test suite for IN LIMIT in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql index 4643605148a0c..1a6c06f9dad49 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql @@ -1,6 +1,9 @@ -- A test suite for multiple columns in predicate in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql index 001c49c460b06..568854ebe2d9b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql @@ -6,6 +6,9 @@ --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN +--CONFIG_DIM2 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM2 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql index e65cb9106c1d4..fa4ae87f041cf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql @@ -1,6 +1,9 @@ -- A test suite for in with cte in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql index 2f6835b59fdd5..e2d4ad522d446 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql @@ -1,5 +1,8 @@ -- Tests NOT-IN subqueries nested inside OR expression(s). +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", 10), (200, "emp 2", NULL), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql index fcdb667ad4523..2d11c5da20633 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -1,6 +1,9 @@ -- A test suite for not-in-joins in parent side, subquery, and both predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql index 8eea84f4f5272..a061e495f51b8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql @@ -4,6 +4,9 @@ -- This file has the same test cases as not-in-unit-tests-multi-column.sql with literals instead of -- subqueries. Small changes have been made to the literals to make them typecheck. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, null), (null, 1.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql index 9f8dc7fca3b94..28ab75121573a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql @@ -15,6 +15,9 @@ -- This can be generalized to include more tests for more columns, but it covers the main cases -- when there is more than one column. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, null), (null, 1.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql index b261363d1dde7..79747022eb1e8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql @@ -4,6 +4,9 @@ -- This file has the same test cases as not-in-unit-tests-single-column.sql with literals instead of -- subqueries. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, 1.0), (2, 3.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql index 2cc08e10acf67..8060246bf3a3f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql @@ -31,6 +31,9 @@ -- cause cases 2, 3, or 4 to be reduced to case 1 by limiting the number of rows returned by the -- subquery, so the row from the parent table should always be included in the output. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, 1.0), (2, 3.0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql index 2748a959cbef8..d8a58afa344db 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql @@ -1,6 +1,9 @@ -- A test suite for simple IN predicate subquery -- It includes correlated cases. +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/timezone.sql b/sql/core/src/test/resources/sql-tests/inputs/timezone.sql new file mode 100644 index 0000000000000..105f2816fb549 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/timezone.sql @@ -0,0 +1,15 @@ +-- valid time zones +SET TIME ZONE 'Asia/Hong_Kong'; +SET TIME ZONE 'GMT+1'; +SET TIME ZONE INTERVAL 10 HOURS; +SET TIME ZONE INTERVAL '15:40:32' HOUR TO SECOND; +SET TIME ZONE LOCAL; + +-- invalid time zone +SET TIME ZONE; +SET TIME ZONE 'invalid/zone'; +SET TIME ZONE INTERVAL 3 DAYS; +SET TIME ZONE INTERVAL 24 HOURS; +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND; +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1'; +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND; diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out index 248157efacde0..18d1a10068794 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out @@ -12,12 +12,12 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn -- !query -select date_format('2018-11-17 13:33:33.333', 'yyyyyyyyyyy') +select date_format('2018-11-17 13:33:33.333', 'yyyyyyy') -- !query schema struct<> -- !query output org.apache.spark.SparkUpgradeException -You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyyyyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out index b7bc448a952a7..b37922b20807d 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out @@ -32,17 +32,17 @@ struct -- !query -select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy yyyyyyy yyyyyyyy yyyyyyyyy yyyyyyyyyy') from v +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v -- !query schema -struct +struct -- !query output -1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 0001582 00001582 000001582 0000001582 -1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 0001969 00001969 000001969 0000001969 -1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 0001970 00001970 000001970 0000001970 -1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 0001996 00001996 000001996 0000001996 -2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 0002018 00002018 000002018 0000002018 -2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 0002019 00002019 000002019 0000002019 -2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 0002100 00002100 000002100 0000002100 +1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 +1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 +1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 +1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 +2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 +2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 +2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out index f724658d354df..5bed88e168f1e 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out @@ -32,17 +32,17 @@ struct -- !query -select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy yyyyyyy yyyyyyyy yyyyyyyyy yyyyyyyyyy') from v +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v -- !query schema -struct +struct -- !query output -1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 0001582 00001582 000001582 0000001582 -1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 0001969 00001969 000001969 0000001969 -1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 0001970 00001970 000001970 0000001970 -1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 0001996 00001996 000001996 0000001996 -2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 0002018 00002018 000002018 0000002018 -2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 0002019 00002019 000002019 0000002019 -2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 0002100 00002100 000002100 0000002100 +1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 +1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 +1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 +1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 +2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 +2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 +2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out index dcee0d4c270a3..c11f5f39bebd0 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -1,5 +1,58 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 13 +-- Number of queries: 19 + + +-- !query +select to_timestamp('294248', 'y') +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select to_timestamp('1', 'yy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('-12', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123', 'yy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('1', 'yyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. + + +-- !query +select to_timestamp('1234567', 'yyyyyyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out index 742e0839d919a..bb7ce74a29ef5 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out @@ -1,5 +1,101 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 13 +-- Number of queries: 25 + + +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'y') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('99', 'yy') +-- !query schema +struct +-- !query output +1999-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('9999', 'yyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('09999', 'yyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out index 742e0839d919a..98146a189a005 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out @@ -1,5 +1,101 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 13 +-- Number of queries: 25 + + +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'y') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('99', 'yy') +-- !query schema +struct +-- !query output +2099-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('9999', 'yyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('09999', 'yyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 36757863ffcb5..79f4d48101ca4 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -308,7 +308,7 @@ Input [2]: [key#x, val#x] (7) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (8) BroadcastHashJoin Left keys [1]: [key#x] @@ -362,7 +362,7 @@ Input [2]: [key#x, val#x] (5) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (6) BroadcastHashJoin Left keys [1]: [key#x] @@ -533,7 +533,7 @@ Input [2]: [key#x, val#x] (7) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (8) BroadcastHashJoin Left keys [1]: [key#x] @@ -643,7 +643,7 @@ Results [2]: [key#x, max(val#x)#x AS max(val)#x] (13) BroadcastExchange Input [2]: [key#x, max(val)#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (14) BroadcastHashJoin Left keys [1]: [key#x] diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 2b07dac0e5d0a..614cb2a137d01 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -316,7 +316,7 @@ Input [2]: [key#x, val#x] (9) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [key#x] @@ -373,7 +373,7 @@ Input [2]: [key#x, val#x] (7) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [key#x] @@ -771,7 +771,7 @@ Input [2]: [key#x, val#x] (9) BroadcastExchange Input [2]: [key#x, val#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [key#x] @@ -853,7 +853,7 @@ Results [2]: [key#x, max(val#x)#x AS max(val)#x] (10) BroadcastExchange Input [2]: [key#x, max(val)#x] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#x] (11) BroadcastHashJoin [codegen id : 4] Left keys [1]: [key#x] diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index d41d25280146b..c349d9d84c226 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 37 +-- Number of queries: 68 -- !query @@ -94,6 +94,62 @@ struct +-- !query output +2 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp +-- !query schema +struct +-- !query output +8 2 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp +-- !query schema +struct +-- !query output +2 2 + + +-- !query +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp +-- !query schema +struct +-- !query output +2450.0 8 2 + + +-- !query +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp +-- !query schema +struct +-- !query output +8 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp +-- !query schema +struct +-- !query output +0 + + -- !query SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a -- !query schema @@ -177,6 +233,68 @@ struct "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 +100 400.0 +20 300.0 +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL 400.0 NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct DATE '2001-01-01')):double,sum(DISTINCT salary) FILTER (WHERE (date_format(CAST(hiredate AS TIMESTAMP), yyyy-MM-dd HH:mm:ss) > 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL NULL NULL + + +-- !query +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01)):double> +-- !query output +10 3 300.0 300.0 +100 2 400.0 400.0 +20 1 300.0 300.0 +30 1 400.0 400.0 +70 1 150.0 150.0 +NULL 1 400.0 NULL + + +-- !query +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b +-- !query schema +struct +-- !query output +1 1 +2 1 +NULL 0 + + -- !query SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1 -- !query schema @@ -261,6 +379,240 @@ struct 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 0 300.0 +30 0 400.0 +70 1 150.0 +NULL 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 0 300.0 +30 1 0 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 1 300.0 300.0 +30 1 1 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 1 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 0 300.0 300.0 +30 1 0 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 0 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 0 1 400.0 +100 2 1 NULL +20 1 0 300.0 +30 1 1 NULL +70 1 1 150.0 +NULL 1 0 NULL + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 0 1 400.0 NULL +100 2 1 NULL 800.0 +20 1 0 300.0 300.0 +30 1 1 NULL 400.0 +70 1 1 150.0 150.0 +NULL 1 0 NULL 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 NULL 2 400.0 +100 1500 2 800.0 +20 320 1 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 1 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 NULL 1 400.0 +100 1500 1 800.0 +20 320 0 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 0 400.0 + + +-- !query +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):double,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 NULL 1 400.0 +100 750.0 1 NULL +20 320.0 0 300.0 +30 430.0 1 NULL +70 870.0 1 150.0 +NULL NULL 0 NULL + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id +-- !query schema +struct 0)):bigint,sum(salary):double> +-- !query output +10 2 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 1 0 400.0 +100 1 1 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + -- !query SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1 -- !query schema @@ -309,6 +661,15 @@ struct<((a + 1) + 1):int,count(b) FILTER (WHERE (b > 0)):bigint> NULL 1 +-- !query +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct 0)):bigint,count(DISTINCT b, c) FILTER (WHERE ((b > 0) AND (c > 2))):bigint> +-- !query output +1 1 + + -- !query SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index fc58bc784c82f..f678e0141eaa9 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -461,7 +461,7 @@ NULL NULL -- !query select width_bucket(5.35, 0.024, 10.06, 5) -- !query schema -struct +struct -- !query output 3 @@ -469,7 +469,7 @@ struct +struct -- !query output 3 @@ -477,7 +477,7 @@ struct +struct -- !query output 3 @@ -485,7 +485,7 @@ struct +struct -- !query output 2 @@ -493,7 +493,7 @@ struct +struct -- !query output NULL @@ -501,7 +501,7 @@ NULL -- !query select width_bucket(null, 0.024, 10.06, 5) -- !query schema -struct +struct -- !query output NULL @@ -509,7 +509,7 @@ NULL -- !query select width_bucket(5.35, null, 10.06, 5) -- !query schema -struct +struct -- !query output NULL @@ -517,7 +517,7 @@ NULL -- !query select width_bucket(5.35, 0.024, null, -5) -- !query schema -struct +struct -- !query output NULL @@ -525,7 +525,7 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, null) -- !query schema -struct +struct -- !query output NULL @@ -533,7 +533,7 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, -5) -- !query schema -struct +struct -- !query output NULL @@ -541,7 +541,7 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L) -- !query schema -struct +struct -- !query output NULL @@ -549,6 +549,6 @@ NULL -- !query select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L - 1) -- !query schema -struct +struct -- !query output 4894746858139549697 diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index ac4e71e244bc0..bb0d452fa04a1 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -276,7 +276,7 @@ PIVOT ( FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) -- !query schema -struct +struct -- !query output 1 15000 NULL 2 NULL 30000 @@ -370,7 +370,7 @@ PIVOT ( FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) ) -- !query schema -struct,[2013, Java]:array> +struct,{2013, Java}:array> -- !query output 2012 [1,1] NULL 2013 NULL [2,2] @@ -404,7 +404,7 @@ PIVOT ( FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 @@ -421,7 +421,7 @@ PIVOT ( FOR s IN ((1, 'a'), (2, 'b')) ) -- !query schema -struct +struct -- !query output 2012 35000 NULL 2013 NULL 78000 @@ -438,7 +438,7 @@ PIVOT ( FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out index 69f96b02782e3..e1f735e5fe1dc 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 4 +-- Number of queries: 5 -- !query @@ -27,6 +27,20 @@ struct 0)):d 2828.9682539682954 +-- !query +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query schema +struct 10)):bigint> +-- !query output +0 NULL +2 NULL +4 NULL +6 NULL +8 NULL + + -- !query select (select count(*) from (values (1)) t0(inner_c)) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out index 7312c20876296..2619634d7d569 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 54 +-- Number of queries: 55 -- !query @@ -443,6 +443,25 @@ struct NULL 1 +-- !query +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten) +-- !query schema +struct +-- !query output +0 NULL +1 NULL +2 NULL +3 NULL +4 NULL +5 NULL +6 NULL +7 NULL +8 NULL +9 NULL +NULL NULL + + -- !query select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col) -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 3cade4f1f9558..d97853d5fc6d0 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -4426,7 +4426,7 @@ struct<> -- !query SELECT width_bucket(5.0, 3.0, 4.0, 0) -- !query schema -struct +struct -- !query output NULL @@ -4434,7 +4434,7 @@ NULL -- !query SELECT width_bucket(5.0, 3.0, 4.0, -5) -- !query schema -struct +struct -- !query output NULL @@ -4442,7 +4442,7 @@ NULL -- !query SELECT width_bucket(3.5, 3.0, 3.0, 888) -- !query schema -struct +struct -- !query output NULL @@ -4450,7 +4450,7 @@ NULL -- !query SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0) -- !query schema -struct +struct -- !query output NULL @@ -4458,7 +4458,7 @@ NULL -- !query SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5) -- !query schema -struct +struct -- !query output NULL @@ -4466,7 +4466,7 @@ NULL -- !query SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888) -- !query schema -struct +struct -- !query output NULL @@ -4474,7 +4474,7 @@ NULL -- !query SELECT width_bucket('NaN', 3.0, 4.0, 888) -- !query schema -struct +struct -- !query output NULL @@ -4482,7 +4482,7 @@ NULL -- !query SELECT width_bucket(double(0), 'NaN', double(4.0), 888) -- !query schema -struct +struct -- !query output NULL @@ -4564,7 +4564,7 @@ struct +struct -- !query output NULL @@ -4572,7 +4572,7 @@ NULL -- !query SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20) -- !query schema -struct +struct -- !query output NULL @@ -4581,7 +4581,7 @@ NULL SELECT width_bucket(double('Infinity'), 1, 10, 10), width_bucket(double('-Infinity'), 1, 10, 10) -- !query schema -struct +struct -- !query output 11 0 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out index 2b1de87a6be5e..76567b689445a 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out @@ -270,7 +270,7 @@ struct +struct -- !query output 0 0 0 0 0 0 @@ -287,7 +287,7 @@ struct +struct -- !query output 0 4 0 1 1 1 @@ -306,7 +306,7 @@ SELECT last(ten) OVER (PARTITION BY four), ten, four FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s ORDER BY four, ten -- !query schema -struct +struct -- !query output 4 0 0 4 0 0 @@ -476,7 +476,7 @@ sum(ten) over (partition by four order by ten), last(ten) over (partition by four order by ten) FROM (select distinct ten, four from tenk1) ss -- !query schema -struct +struct -- !query output 0 0 0 0 0 2 2 2 @@ -506,7 +506,7 @@ sum(ten) over (partition by four order by ten range between unbounded preceding last(ten) over (partition by four order by ten range between unbounded preceding and current row) FROM (select distinct ten, four from tenk1) ss -- !query schema -struct +struct -- !query output 0 0 0 0 0 2 2 2 @@ -536,7 +536,7 @@ sum(ten) over (partition by four order by ten range between unbounded preceding last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) FROM (select distinct ten, four from tenk1) ss -- !query schema -struct +struct -- !query output 0 0 20 8 0 2 20 8 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index f41659a196ae1..ccddf9db172a6 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -101,7 +101,7 @@ from window w as (order by ss.id asc nulls first range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 1 3 2 2 1 4 @@ -123,7 +123,7 @@ from window w as (order by ss.id asc nulls last range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 1 3 2 2 1 4 @@ -145,7 +145,7 @@ from window w as (order by ss.id desc nulls first range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 3 1 2 2 4 1 @@ -167,7 +167,7 @@ from window w as (order by ss.id desc nulls last range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 3 1 2 2 4 1 @@ -182,7 +182,7 @@ NULL 43 42 43 select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) from range(32764, 32767) x -- !query schema -struct +struct -- !query output 32764 32766 32765 32766 @@ -193,7 +193,7 @@ struct +struct -- !query output -32766 -32766 @@ -202,7 +202,7 @@ struct +struct -- !query output 2147483644 2147483646 2147483645 2147483646 @@ -213,7 +213,7 @@ struct +struct -- !query output -2147483646 -2147483646 @@ -272,7 +272,7 @@ from numerics window w as (order by f_float4 range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -289,7 +289,7 @@ from numerics window w as (order by f_float4 range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -306,7 +306,7 @@ from numerics window w as (order by f_float4 range between 'inf' preceding and 'inf' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 1 7 @@ -323,7 +323,7 @@ from numerics window w as (order by f_float4 range between 1.1 preceding and 'NaN' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 2 7 @@ -340,7 +340,7 @@ from numerics window w as (order by f_float8 range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -357,7 +357,7 @@ from numerics window w as (order by f_float8 range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -374,7 +374,7 @@ from numerics window w as (order by f_float8 range between 'inf' preceding and 'inf' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 1 7 @@ -391,7 +391,7 @@ from numerics window w as (order by f_float8 range between 1.1 preceding and 'NaN' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 2 7 @@ -408,7 +408,7 @@ from numerics window w as (order by f_numeric range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3 1 1 2 -1 2 3 @@ -425,7 +425,7 @@ from numerics window w as (order by f_numeric range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3 1 1 2 -1 2 3 @@ -442,7 +442,7 @@ from numerics window w as (order by f_numeric range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3 1 1 2 -1 2 3 diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index c92c1ddca774f..2eef926f63e37 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 8 +-- Number of queries: 30 -- !query @@ -37,6 +37,31 @@ java.lang.IllegalArgumentException Regex group count is 0, but the specified group index is 2 +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output + + + -- !query SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') -- !query schema @@ -67,3 +92,163 @@ SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2) struct -- !query output a + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 2, but the specified group index is 3 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 2 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["1","","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct> +-- !query output +["1a","2b","14m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct> +-- !query output +["a","b","m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 2, but the specified group index is 3 + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +The specified group index cannot be less than zero + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["","2","14"] diff --git a/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out index 6984b34c365ec..eaaf894590d35 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out @@ -91,7 +91,7 @@ struct<> -- !query -CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1 +CREATE TEMPORARY VIEW tv AS SELECT 1 AS c1 -- !query schema struct<> -- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out new file mode 100644 index 0000000000000..d816043130737 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out @@ -0,0 +1,135 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query +SET TIME ZONE 'Asia/Hong_Kong' +-- !query schema +struct +-- !query output +spark.sql.session.timeZone Asia/Hong_Kong + + +-- !query +SET TIME ZONE 'GMT+1' +-- !query schema +struct +-- !query output +spark.sql.session.timeZone GMT+1 + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS +-- !query schema +struct +-- !query output +spark.sql.session.timeZone +10:00 + + +-- !query +SET TIME ZONE INTERVAL '15:40:32' HOUR TO SECOND +-- !query schema +struct +-- !query output +spark.sql.session.timeZone +15:40:32 + + +-- !query +SET TIME ZONE LOCAL +-- !query schema +struct +-- !query output +spark.sql.session.timeZone America/Los_Angeles + + +-- !query +SET TIME ZONE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Invalid time zone displacement value(line 1, pos 0) + +== SQL == +SET TIME ZONE +^^^ + + +-- !query +SET TIME ZONE 'invalid/zone' +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Cannot resolve the given timezone with ZoneId.of(_, ZoneId.SHORT_IDS) + + +-- !query +SET TIME ZONE INTERVAL 3 DAYS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL 3 DAYS +--------------^^^ + + +-- !query +SET TIME ZONE INTERVAL 24 HOURS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL 24 HOURS +--------------^^^ + + +-- !query +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND +--------------^^^ + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Invalid time zone displacement value(line 1, pos 0) + +== SQL == +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' +^^^ + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) + +== SQL == +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND +--------------^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out index 087b4ed9302d8..414435e6b781d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -276,7 +276,7 @@ PIVOT ( FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) -- !query schema -struct +struct -- !query output 1 15000 NULL 2 NULL 30000 @@ -370,7 +370,7 @@ PIVOT ( FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 @@ -387,7 +387,7 @@ PIVOT ( FOR s IN ((1, 'a'), (2, 'b')) ) -- !query schema -struct +struct -- !query output 2012 35000 NULL 2013 NULL 78000 @@ -404,7 +404,7 @@ PIVOT ( FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) -- !query schema -struct +struct -- !query output 2012 15000 NULL 2013 NULL 30000 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 131ab1b94f59e..36c10874cc183 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -20,11 +20,9 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} import java.util.Locale -import scala.collection.JavaConverters._ - import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} -import org.scalatest.Matchers._ +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.sql.catalyst.expressions.{InSet, Literal, NamedExpression} import org.apache.spark.sql.execution.ProjectExec @@ -1422,4 +1420,36 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { }.getMessage should include("No such struct field b in a, B") } } + + test("withField user-facing examples") { + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".withField("c", lit(3))), + Row(Row(1, 2, 3))) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".withField("b", lit(3))), + Row(Row(1, 3))) + + checkAnswer( + sql("SELECT CAST(NULL AS struct) struct_col") + .select($"struct_col".withField("c", lit(3))), + Row(null)) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col") + .select($"struct_col".withField("b", lit(100))), + Row(Row(1, 100, 100))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3))), + Row(Row(Row(1, 2, 3)))) + + intercept[AnalysisException] { + sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3))) + }.getMessage should include("Ambiguous reference to fields") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 09f30bb5e2c77..e954e2bf1c46d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.util.Random -import org.scalatest.Matchers.the +import org.scalatest.matchers.must.Matchers.the import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -31,7 +31,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.DecimalData import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) @@ -1037,6 +1036,13 @@ class DataFrameAggregateSuite extends QueryTest val groupBy = df.groupBy("b").agg(count("*")) checkAnswer(groupBy, Row(null, 1) :: Row(Row(null), 1) :: Row(Row(1.0), 1) :: Nil) } + + test("SPARK-32344: Unevaluable's set to FIRST/LAST ignoreNullsExpr in distinct aggregates") { + val queryTemplate = (agg: String) => + s"SELECT $agg(DISTINCT v) FROM (SELECT v FROM VALUES 1, 2, 3 t(v) ORDER BY v)" + checkAnswer(sql(queryTemplate("FIRST")), Row(1)) + checkAnswer(sql(queryTemplate("LAST")), Row(3)) + } } case class B(c: Option[Double]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index cb11519497747..585a835024e19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -936,10 +936,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val answer = Seq(Row(Seq(2, 3)), Row(Seq(5))) checkAnswer(df.select(slice(df("x"), 2, 2)), answer) + checkAnswer(df.select(slice(df("x"), lit(2), lit(2))), answer) checkAnswer(df.selectExpr("slice(x, 2, 2)"), answer) val answerNegative = Seq(Row(Seq(3)), Row(Seq(5))) checkAnswer(df.select(slice(df("x"), -1, 1)), answerNegative) + checkAnswer(df.select(slice(df("x"), lit(-1), lit(1))), answerNegative) checkAnswer(df.selectExpr("slice(x, -1, 1)"), answerNegative) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 19601726fa393..8078165cf8ec9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.util.Random -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.stat.StatFunctions diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 52ef5895ed9ad..bcfc77545bbd2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -26,7 +26,8 @@ import java.util.concurrent.atomic.AtomicLong import scala.reflect.runtime.universe.TypeTag import scala.util.Random -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} @@ -1267,7 +1268,7 @@ class DataFrameSuite extends QueryTest s"""+----------------+ || a| |+----------------+ - ||[1 -> a, 2 -> b]| + ||{1 -> a, 2 -> b}| |+----------------+ |""".stripMargin) val df3 = Seq(((1, "a"), 0), ((2, "b"), 0)).toDF("a", "b") @@ -1275,8 +1276,8 @@ class DataFrameSuite extends QueryTest s"""+------+---+ || a| b| |+------+---+ - ||[1, a]| 0| - ||[2, b]| 0| + ||{1, a}| 0| + ||{2, b}| 0| |+------+---+ |""".stripMargin) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 09ce43e4e2b27..bc6adfb857b02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.scalatest.Matchers.the +import org.scalatest.matchers.must.Matchers.the import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.optimizer.TransposeWindow diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala index 43de2663b1236..b20d050f2fc4a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala @@ -34,7 +34,7 @@ class DatasetSerializerRegistratorSuite extends QueryTest with SharedSparkSessio override protected def sparkConf: SparkConf = { // Make sure we use the KryoRegistrator - super.sparkConf.set(KRYO_USER_REGISTRATORS, TestRegistrator().getClass.getCanonicalName) + super.sparkConf.set(KRYO_USER_REGISTRATORS, Seq(TestRegistrator().getClass.getCanonicalName)) } test("Kryo registrator") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 06600c1e4b1d7..4923e8b556907 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1116,8 +1116,8 @@ class DatasetSuite extends QueryTest """+--------+ || f| |+--------+ - ||[foo, 1]| - ||[bar, 2]| + ||{foo, 1}| + ||{bar, 2}| |+--------+ |""".stripMargin diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index cdf9ea4b31ee7..8edfb91d15fff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -396,7 +396,7 @@ abstract class DynamicPartitionPruningSuiteBase """.stripMargin) val found = df.queryExecution.executedPlan.find { - case BroadcastHashJoinExec(_, _, p: ExistenceJoin, _, _, _, _) => true + case BroadcastHashJoinExec(_, _, p: ExistenceJoin, _, _, _, _, _) => true case _ => false } @@ -1240,6 +1240,30 @@ abstract class DynamicPartitionPruningSuiteBase } } + test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + + "canonicalization and exchange reuse") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql( + """ WITH view1 as ( + | SELECT f.store_id FROM fact_stats f WHERE f.units_sold = 70 + | ) + | + | SELECT * FROM view1 v1 join view1 v2 WHERE v1.store_id = v2.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + val reuseExchangeNodes = df.queryExecution.executedPlan.collect { + case se: ReusedExchangeExec => se + } + assert(reuseExchangeNodes.size == 1, "Expected plan to contain 1 ReusedExchangeExec " + + s"nodes. Found ${reuseExchangeNodes.size}") + + checkAnswer(df, Row(15, 15) :: Nil) + } + } + } + test("Plan broadcast pruning only when the broadcast can be reused") { Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 70303792fdf1a..e43a4147ceb63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -347,7 +347,7 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite test("Coalesced bucket info should be a part of explain string") { withTable("t1", "t2") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { Seq(1, 2).toDF("i").write.bucketBy(8, "i").saveAsTable("t1") Seq(2, 3).toDF("i").write.bucketBy(4, "i").saveAsTable("t2") val df1 = spark.table("t1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 231a8f2aa7ddd..e9bff64d72fc3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -633,13 +633,15 @@ class FileBasedDataSourceSuite extends QueryTest assert(fileList.toSet === expectedFileList.toSet) - val fileList2 = spark.read.format("binaryFile") - .option("recursiveFileLookup", true) - .option("pathGlobFilter", "*.bin") - .load(dataPath) - .select("path").collect().map(_.getString(0)) - - assert(fileList2.toSet === expectedFileList.filter(_.endsWith(".bin")).toSet) + withClue("SPARK-32368: 'recursiveFileLookup' and 'pathGlobFilter' can be case insensitive") { + val fileList2 = spark.read.format("binaryFile") + .option("RecuRsivefileLookup", true) + .option("PaThglobFilter", "*.bin") + .load(dataPath) + .select("path").collect().map(_.getString(0)) + + assert(fileList2.toSet === expectedFileList.filter(_.endsWith(".bin")).toSet) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index 6391d5664d5c6..80346b350c142 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -197,7 +197,7 @@ object IntegratedUDFTestUtils extends SQLHelper { lazy val pythonExec: String = { val pythonExec = sys.env.getOrElse( - "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3.6")) + "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3")) if (TestUtils.testCommandAvailable(pythonExec)) { pythonExec } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index f24da6df67ca0..bedfbffc789ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrd import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf @@ -88,6 +89,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan if (operators.head.getClass != c) { fail(s"$sqlString expected operator: $c, but got ${operators.head}\n physical: \n$physical") } + operators.head } test("join operator selection") { @@ -1086,4 +1088,104 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan assert(df2.join(df1, "id").collect().isEmpty) } } + + test("SPARK-32330: Preserve shuffled hash join build side partitioning") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + val df1 = spark.range(10).select($"id".as("k1")) + val df2 = spark.range(30).select($"id".as("k2")) + Seq("inner", "cross").foreach(joinType => { + val plan = df1.join(df2, $"k1" === $"k2", joinType).groupBy($"k1").count() + .queryExecution.executedPlan + assert(plan.collect { case _: ShuffledHashJoinExec => true }.size === 1) + // No extra shuffle before aggregate + assert(plan.collect { case _: ShuffleExchangeExec => true }.size === 2) + }) + } + } + + test("SPARK-32383: Preserve hash join (BHJ and SHJ) stream side ordering") { + val df1 = spark.range(100).select($"id".as("k1")) + val df2 = spark.range(100).select($"id".as("k2")) + val df3 = spark.range(3).select($"id".as("k3")) + val df4 = spark.range(100).select($"id".as("k4")) + + // Test broadcast hash join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50") { + Seq("inner", "left_outer").foreach(joinType => { + val plan = df1.join(df2, $"k1" === $"k2", joinType) + .join(df3, $"k1" === $"k3", joinType) + .join(df4, $"k1" === $"k4", joinType) + .queryExecution + .executedPlan + assert(plan.collect { case _: SortMergeJoinExec => true }.size === 2) + assert(plan.collect { case _: BroadcastHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join + assert(plan.collect { case _: SortExec => true }.size === 3) + }) + } + + // Test shuffled hash join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + val df3 = spark.range(10).select($"id".as("k3")) + + Seq("inner", "left_outer").foreach(joinType => { + val plan = df1.join(df2, $"k1" === $"k2", joinType) + .join(df3, $"k1" === $"k3", joinType) + .join(df4, $"k1" === $"k4", joinType) + .queryExecution + .executedPlan + assert(plan.collect { case _: SortMergeJoinExec => true }.size === 2) + assert(plan.collect { case _: ShuffledHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join + assert(plan.collect { case _: SortExec => true }.size === 3) + }) + } + } + + test("SPARK-32290: SingleColumn Null Aware Anti Join Optimize") { + withSQLConf(SQLConf.OPTIMIZE_NULL_AWARE_ANTI_JOIN.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString) { + // positive not in subquery case + var joinExec = assertJoin(( + "select * from testData where key not in (select a from testData2)", + classOf[BroadcastHashJoinExec])) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // negative not in subquery case since multi-column is not supported + assertJoin(( + "select * from testData where (key, key + 1) not in (select * from testData2)", + classOf[BroadcastNestedLoopJoinExec])) + + // positive hand-written left anti join + // testData.key nullable false + // testData3.b nullable true + joinExec = assertJoin(( + "select * from testData left anti join testData3 ON key = b or isnull(key = b)", + classOf[BroadcastHashJoinExec])) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // negative hand-written left anti join + // testData.key nullable false + // testData2.a nullable false + // isnull(key = a) will be optimized to true literal and removed + joinExec = assertJoin(( + "select * from testData left anti join testData2 ON key = a or isnull(key = a)", + classOf[BroadcastHashJoinExec])) + assert(!joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // negative hand-written left anti join + // not match pattern Or(EqualTo(a=b), IsNull(EqualTo(a=b)) + assertJoin(( + "select * from testData2 left anti join testData3 ON testData2.a = testData3.b or " + + "isnull(testData2.b = testData3.b)", + classOf[BroadcastNestedLoopJoinExec])) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala new file mode 100644 index 0000000000000..78b314272aa07 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/NestedDataSourceSuite.scala @@ -0,0 +1,88 @@ +/* + * 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 + +import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{LongType, StructType} + +// Datasource tests for nested schemas +trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession { + protected val nestedDataSources: Seq[String] = Seq("orc", "parquet", "json") + protected def readOptions(schema: StructType): Map[String, String] = Map.empty + protected def save(selectExpr: Seq[String], format: String, path: String): Unit = { + spark + .range(1L) + .selectExpr(selectExpr: _*) + .write.mode("overwrite") + .format(format) + .save(path) + } + protected val colType: String = "in the data schema" + + test("SPARK-32431: consistent error for nested and top-level duplicate columns") { + Seq( + Seq("id AS lowercase", "id + 1 AS camelCase") -> + new StructType() + .add("LowerCase", LongType) + .add("camelcase", LongType) + .add("CamelCase", LongType), + Seq("NAMED_STRUCT('lowercase', id, 'camelCase', id + 1) AS StructColumn") -> + new StructType().add("StructColumn", + new StructType() + .add("LowerCase", LongType) + .add("camelcase", LongType) + .add("CamelCase", LongType)) + ).foreach { case (selectExpr: Seq[String], caseInsensitiveSchema: StructType) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + nestedDataSources.map { format => + withClue(s"format = $format select = ${selectExpr.mkString(",")}") { + withTempPath { dir => + val path = dir.getCanonicalPath + save(selectExpr, format, path) + val e = intercept[AnalysisException] { + spark + .read + .options(readOptions(caseInsensitiveSchema)) + .schema(caseInsensitiveSchema) + .format(format) + .load(path) + .show + } + assert(e.getMessage.contains(s"Found duplicate column(s) $colType: `camelcase`")) + } + } + } + } + } + } +} + +class NestedDataSourceV1Suite extends NestedDataSourceSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, nestedDataSources.mkString(",")) +} + +class NestedDataSourceV2Suite extends NestedDataSourceSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 989f304b1f07f..64d2567f84f1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3560,6 +3560,40 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-32237: Hint in CTE") { + withTable("t") { + sql("CREATE TABLE t USING PARQUET AS SELECT 1 AS id") + checkAnswer( + sql(s""" + |WITH cte AS (SELECT /*+ REPARTITION(3) */ * FROM t) + |SELECT * FROM cte + """.stripMargin), + Row(1) :: Nil) + } + } + + test("SPARK-32372: ResolveReferences.dedupRight should only rewrite attributes for ancestor " + + "plans of the conflict plan") { + sql("SELECT name, avg(age) as avg_age FROM person GROUP BY name") + .createOrReplaceTempView("person_a") + sql("SELECT p1.name, p2.avg_age FROM person p1 JOIN person_a p2 ON p1.name = p2.name") + .createOrReplaceTempView("person_b") + sql("SELECT * FROM person_a UNION SELECT * FROM person_b") + .createOrReplaceTempView("person_c") + checkAnswer( + sql("SELECT p1.name, p2.avg_age FROM person_c p1 JOIN person_c p2 ON p1.name = p2.name"), + Row("jim", 20.0) :: Row("mike", 30.0) :: Nil) + } + + test("SPARK-32280: Avoid duplicate rewrite attributes when there're multiple JOINs") { + sql("SELECT 1 AS id").createOrReplaceTempView("A") + sql("SELECT id, 'foo' AS kind FROM A").createOrReplaceTempView("B") + sql("SELECT l.id as id FROM B AS l LEFT SEMI JOIN B AS r ON l.kind = r.kind") + .createOrReplaceTempView("C") + checkAnswer(sql("SELECT 0 FROM ( SELECT * FROM B JOIN C USING (id)) " + + "JOIN ( SELECT * FROM B JOIN C USING (id)) USING (id)"), Row(0)) + } } case class Foo(bar: Option[String]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index e914d8398e921..9da32d02aa723 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf._ @@ -257,4 +258,27 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { context.stop() } } + + test("SPARK-32160: Disallow to create SparkSession in executors") { + val session = SparkSession.builder().master("local-cluster[3, 1, 1024]").getOrCreate() + + val error = intercept[SparkException] { + session.range(1).foreach { v => + SparkSession.builder.master("local").getOrCreate() + () + } + }.getMessage() + + assert(error.contains("SparkSession should only be created and accessed on the driver.")) + } + + test("SPARK-32160: Allow to create SparkSession in executors if the config is set") { + val session = SparkSession.builder().master("local-cluster[3, 1, 1024]").getOrCreate() + + session.range(1).foreach { v => + SparkSession.builder.master("local") + .config(EXECUTOR_ALLOW_SPARK_CONTEXT.key, true).getOrCreate().stop() + () + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index d9c90c7dbd085..e5e8bc6917799 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -16,17 +16,24 @@ */ package org.apache.spark.sql -import java.util.Locale +import java.util.{Locale, UUID} -import org.apache.spark.{SparkFunSuite, TaskContext} +import scala.concurrent.Future + +import org.apache.spark.{MapOutputStatistics, SparkFunSuite, TaskContext} +import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, UnresolvedHint} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Statistics, UnresolvedHint} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, BroadcastExchangeLike, ShuffleExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.COLUMN_BATCH_SIZE @@ -145,33 +152,83 @@ class SparkSessionExtensionSuite extends SparkFunSuite { } } - test("inject columnar") { + test("inject adaptive query prep rule") { val extensions = create { extensions => + // inject rule that will run during AQE query stage preparation and will add custom tags + // to the plan + extensions.injectQueryStagePrepRule(session => MyQueryStagePrepRule()) + // inject rule that will run during AQE query stage optimization and will verify that the + // custom tags were written in the preparation phase extensions.injectColumnar(session => - MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) + MyColumarRule(MyNewQueryStageRule(), MyNewQueryStageRule())) } withSession(extensions) { session => - // The ApplyColumnarRulesAndInsertTransitions rule is not applied when enable AQE - session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, false) + session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, true) + assert(session.sessionState.queryStagePrepRules.contains(MyQueryStagePrepRule())) assert(session.sessionState.columnarRules.contains( - MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + MyColumarRule(MyNewQueryStageRule(), MyNewQueryStageRule()))) import session.sqlContext.implicits._ - // repartitioning avoids having the add operation pushed up into the LocalTableScan val data = Seq((100L), (200L), (300L)).toDF("vals").repartition(1) val df = data.selectExpr("vals + 1") - // Verify that both pre and post processing of the plan worked. - val found = df.queryExecution.executedPlan.collect { - case rep: ReplacedRowToColumnarExec => 1 - case proj: ColumnarProjectExec => 10 - case c2r: ColumnarToRowExec => 100 - }.sum - assert(found == 111) + df.collect() + } + } + + test("inject columnar AQE on") { + testInjectColumnar(true) + } + + test("inject columnar AQE off") { + testInjectColumnar(false) + } + + private def testInjectColumnar(enableAQE: Boolean): Unit = { + def collectPlanSteps(plan: SparkPlan): Seq[Int] = plan match { + case a: AdaptiveSparkPlanExec => + assert(a.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) + collectPlanSteps(a.executedPlan) + case _ => plan.collect { + case _: ReplacedRowToColumnarExec => 1 + case _: ColumnarProjectExec => 10 + case _: ColumnarToRowExec => 100 + case s: QueryStageExec => collectPlanSteps(s.plan).sum + case _: MyShuffleExchangeExec => 1000 + case _: MyBroadcastExchangeExec => 10000 + } + } + + val extensions = create { extensions => + extensions.injectColumnar(session => + MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) + } + withSession(extensions) { session => + session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, enableAQE) + assert(session.sessionState.columnarRules.contains( + MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + import session.sqlContext.implicits._ + // perform a join to inject a broadcast exchange + val left = Seq((1, 50L), (2, 100L), (3, 150L)).toDF("l1", "l2") + val right = Seq((1, 50L), (2, 100L), (3, 150L)).toDF("r1", "r2") + val data = left.join(right, $"l1" === $"r1") + // repartitioning avoids having the add operation pushed up into the LocalTableScan + .repartition(1) + val df = data.selectExpr("l2 + r2") + // execute the plan so that the final adaptive plan is available when AQE is on + df.collect() + val found = collectPlanSteps(df.queryExecution.executedPlan).sum + // 1 MyBroadcastExchangeExec + // 1 MyShuffleExchangeExec + // 1 ColumnarToRowExec + // 2 ColumnarProjectExec + // 1 ReplacedRowToColumnarExec + // so 11121 is expected. + assert(found == 11121) // Verify that we get back the expected, wrong, result val result = df.collect() - assert(result(0).getLong(0) == 102L) // Check that broken columnar Add was used. - assert(result(1).getLong(0) == 202L) - assert(result(2).getLong(0) == 302L) + assert(result(0).getLong(0) == 101L) // Check that broken columnar Add was used. + assert(result(1).getLong(0) == 201L) + assert(result(2).getLong(0) == 301L) } } @@ -671,6 +728,16 @@ case class PreRuleReplaceAddWithBrokenVersion() extends Rule[SparkPlan] { def replaceWithColumnarPlan(plan: SparkPlan): SparkPlan = try { plan match { + case e: ShuffleExchangeExec => + // note that this is not actually columnar but demonstrates that exchanges can + // be replaced. + val replaced = e.withNewChildren(e.children.map(replaceWithColumnarPlan)) + MyShuffleExchangeExec(replaced.asInstanceOf[ShuffleExchangeExec]) + case e: BroadcastExchangeExec => + // note that this is not actually columnar but demonstrates that exchanges can + // be replaced. + val replaced = e.withNewChildren(e.children.map(replaceWithColumnarPlan)) + MyBroadcastExchangeExec(replaced.asInstanceOf[BroadcastExchangeExec]) case plan: ProjectExec => new ColumnarProjectExec(plan.projectList.map((exp) => replaceWithColumnarExpression(exp).asInstanceOf[NamedExpression]), @@ -689,6 +756,41 @@ case class PreRuleReplaceAddWithBrokenVersion() extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = replaceWithColumnarPlan(plan) } +/** + * Custom Exchange used in tests to demonstrate that shuffles can be replaced regardless of + * whether AQE is enabled. + */ +case class MyShuffleExchangeExec(delegate: ShuffleExchangeExec) extends ShuffleExchangeLike { + override def numMappers: Int = delegate.numMappers + override def numPartitions: Int = delegate.numPartitions + override def canChangeNumPartitions: Boolean = delegate.canChangeNumPartitions + override def mapOutputStatisticsFuture: Future[MapOutputStatistics] = + delegate.mapOutputStatisticsFuture + override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] = + delegate.getShuffleRDD(partitionSpecs) + override def runtimeStatistics: Statistics = delegate.runtimeStatistics + override def child: SparkPlan = delegate.child + override protected def doExecute(): RDD[InternalRow] = delegate.execute() + override def outputPartitioning: Partitioning = delegate.outputPartitioning +} + +/** + * Custom Exchange used in tests to demonstrate that broadcasts can be replaced regardless of + * whether AQE is enabled. + */ +case class MyBroadcastExchangeExec(delegate: BroadcastExchangeExec) extends BroadcastExchangeLike { + override def runId: UUID = delegate.runId + override def relationFuture: java.util.concurrent.Future[Broadcast[Any]] = + delegate.relationFuture + override def completionFuture: Future[Broadcast[Any]] = delegate.completionFuture + override def runtimeStatistics: Statistics = delegate.runtimeStatistics + override def child: SparkPlan = delegate.child + override protected def doPrepare(): Unit = delegate.prepare() + override protected def doExecute(): RDD[InternalRow] = delegate.execute() + override def doExecuteBroadcast[T](): Broadcast[T] = delegate.executeBroadcast() + override def outputPartitioning: Partitioning = delegate.outputPartitioning +} + class ReplacedRowToColumnarExec(override val child: SparkPlan) extends RowToColumnarExec(child) { @@ -731,6 +833,31 @@ class MyExtensions extends (SparkSessionExtensions => Unit) { } } +object QueryPrepRuleHelper { + val myPrepTag: TreeNodeTag[String] = TreeNodeTag[String]("myPrepTag") + val myPrepTagValue: String = "myPrepTagValue" +} + +// this rule will run during AQE query preparation and will write custom tags to each node +case class MyQueryStagePrepRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case plan => + plan.setTagValue(QueryPrepRuleHelper.myPrepTag, QueryPrepRuleHelper.myPrepTagValue) + plan + } +} + +// this rule will run during AQE query stage optimization and will verify custom tags were +// already written during query preparation phase +case class MyNewQueryStageRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case plan if !plan.isInstanceOf[AdaptiveSparkPlanExec] => + assert(plan.getTagValue(QueryPrepRuleHelper.myPrepTag).get == + QueryPrepRuleHelper.myPrepTagValue) + plan + } +} + case class MyRule2(spark: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan } 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 f904b53fe47eb..8d5166b5398cc 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 @@ -154,8 +154,25 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { Row("300", "100") :: Row("400", "100") :: Row("400-400", "100") :: Nil) } + test("string regex_extract_all") { + val df = Seq( + ("100-200,300-400", "(\\d+)-(\\d+)"), + ("101-201,301-401", "(\\d+)-(\\d+)"), + ("102-202,302-402", "(\\d+)")).toDF("a", "b") + + checkAnswer( + df.select( + regexp_extract_all($"a", "(\\d+)-(\\d+)", 1), + regexp_extract_all($"a", "(\\d+)-(\\d+)", 2)), + Row(Seq("100", "300"), Seq("200", "400")) :: + Row(Seq("101", "301"), Seq("201", "401")) :: + Row(Seq("102", "302"), Seq("202", "402")) :: Nil) + } + test("non-matching optional group") { val df = Seq(Tuple1("aaaac")).toDF("s") + + // regexp_extract checkAnswer( df.select(regexp_extract($"s", "(foo)", 1)), Row("") @@ -164,6 +181,16 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { df.select(regexp_extract($"s", "(a+)(b)?(c)", 2)), Row("") ) + + // regexp_extract_all + checkAnswer( + df.select(regexp_extract_all($"s", "(foo)", 1)), + Row(Seq()) + ) + checkAnswer( + df.select(regexp_extract_all($"s", "(a+)(b)?(c)", 2)), + Row(Seq("")) + ) } test("string ascii function") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 2bb9aa55e4579..a21c461e84588 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD +import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -1646,4 +1647,110 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(df, df2) checkAnswer(df, Nil) } + + test("SPARK-32290: SingleColumn Null Aware Anti Join Optimize") { + Seq(true, false).foreach { enableNAAJ => + Seq(true, false).foreach { enableAQE => + Seq(true, false).foreach { enableCodegen => + withSQLConf( + SQLConf.OPTIMIZE_NULL_AWARE_ANTI_JOIN.key -> enableNAAJ.toString, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> enableAQE.toString, + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> enableCodegen.toString) { + + def findJoinExec(df: DataFrame): BaseJoinExec = { + df.queryExecution.sparkPlan.collectFirst { + case j: BaseJoinExec => j + }.get + } + + var df: DataFrame = null + var joinExec: BaseJoinExec = null + + // single column not in subquery -- empty sub-query + df = sql("select * from l where a not in (select c from r where c > 10)") + checkAnswer(df, spark.table("l")) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- sub-query include null + df = sql("select * from l where a not in (select c from r where d < 6.0)") + checkAnswer(df, Seq.empty) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- streamedSide row is null + df = + sql("select * from l where b = 5.0 and a not in(select c from r where c is not null)") + checkAnswer(df, Seq.empty) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- streamedSide row is not null, match found + df = + sql("select * from l where a = 6 and a not in (select c from r where c is not null)") + checkAnswer(df, Seq.empty) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- streamedSide row is not null, match not found + df = + sql("select * from l where a = 1 and a not in (select c from r where c is not null)") + checkAnswer(df, Row(1, 2.0) :: Row(1, 2.0) :: Nil) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // single column not in subquery -- d = b + 10 joinKey found, match ExtractEquiJoinKeys + df = sql("select * from l where a not in (select c from r where d = b + 10)") + checkAnswer(df, spark.table("l")) + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(!joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + + // single column not in subquery -- d = b + 10 and b = 5.0 => d = 15, joinKey not found + // match ExtractSingleColumnNullAwareAntiJoin + df = + sql("select * from l where b = 5.0 and a not in (select c from r where d = b + 10)") + checkAnswer(df, Row(null, 5.0) :: Nil) + if (enableNAAJ) { + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + } else { + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + + // multi column not in subquery + df = sql("select * from l where (a, b) not in (select c, d from r where c > 10)") + checkAnswer(df, spark.table("l")) + assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) + } + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index f0d5a61ad8006..04af7d1a68682 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -22,6 +22,8 @@ import java.sql.Timestamp import java.time.{Instant, LocalDate} import java.time.format.DateTimeFormatter +import scala.collection.mutable.{ArrayBuffer, WrappedArray} + import org.apache.spark.SparkException import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -548,7 +550,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { udf((d: LocalDate, i: Instant) => LocalDateInstantType(d, i))) checkAnswer(df.selectExpr(s"buildLocalDateInstantType(d, i) as di") .select('di.cast(StringType)), - Row(s"[$expectedDate, $expectedInstant]") :: Nil) + Row(s"{$expectedDate, $expectedInstant}") :: Nil) // test null cases spark.udf.register("buildLocalDateInstantType", @@ -578,7 +580,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { udf((t: Timestamp, i: Instant) => TimestampInstantType(t, i))) checkAnswer(df.selectExpr("buildTimestampInstantType(t, i) as ti") .select('ti.cast(StringType)), - Row(s"[$expectedTimestamp, $expectedInstant]")) + Row(s"{$expectedTimestamp, $expectedInstant}")) // test null cases spark.udf.register("buildTimestampInstantType", @@ -787,4 +789,13 @@ class UDFSuite extends QueryTest with SharedSparkSession { Seq(Array(1)).toDF("a").createOrReplaceTempView("t") checkAnswer(sql("SELECT key(a) AS k FROM t GROUP BY key(a)"), Row(1) :: Nil) } + + test("SPARK-32459: UDF should not fail on WrappedArray") { + val myUdf = udf((a: WrappedArray[Int]) => + WrappedArray.make[Int](Array(a.head + 99))) + checkAnswer(Seq(Array(1)) + .toDF("col") + .select(myUdf(Column("col"))), + Row(ArrayBuffer(100))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 85aea3ce41ecc..ffc115e6b7600 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2185,7 +2185,7 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql("DESCRIBE FUNCTION testcat.ns1.ns2.fun") } - assert(e.message.contains("DESCRIBE FUNCTION is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) val e1 = intercept[AnalysisException] { sql("DESCRIBE FUNCTION default.ns1.ns2.fun") @@ -2200,14 +2200,14 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql(s"SHOW FUNCTIONS LIKE $function") } - assert(e.message.contains("SHOW FUNCTIONS is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) } test("DROP FUNCTION: only support session catalog") { val e = intercept[AnalysisException] { sql("DROP FUNCTION testcat.ns1.ns2.fun") } - assert(e.message.contains("DROP FUNCTION is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) val e1 = intercept[AnalysisException] { sql("DROP FUNCTION default.ns1.ns2.fun") @@ -2220,7 +2220,7 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql("CREATE FUNCTION testcat.ns1.ns2.fun as 'f'") } - assert(e.message.contains("CREATE FUNCTION is only supported in v1 catalog")) + assert(e.message.contains("function is only supported in v1 catalog")) val e1 = intercept[AnalysisException] { sql("CREATE FUNCTION default.ns1.ns2.fun as 'f'") @@ -2229,6 +2229,19 @@ class DataSourceV2SQLSuite "The namespace in session catalog must have exactly one name part: default.ns1.ns2.fun")) } + test("REFRESH FUNCTION: only support session catalog") { + val e = intercept[AnalysisException] { + sql("REFRESH FUNCTION testcat.ns1.ns2.fun") + } + assert(e.message.contains("function is only supported in v1 catalog")) + + val e1 = intercept[AnalysisException] { + sql("REFRESH FUNCTION default.ns1.ns2.fun") + } + assert(e1.message.contains( + "The namespace in session catalog must have exactly one name part: default.ns1.ns2.fun")) + } + test("global temp view should not be masked by v2 catalog") { val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) spark.conf.set(s"spark.sql.catalog.$globalTempDB", classOf[InMemoryTableCatalog].getName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 575efec364812..2e25c40cecd78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.internal.SQLConf.MAX_NESTED_VIEW_DEPTH import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -266,6 +267,16 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } + test("SPARK-32374: disallow setting properties for CREATE TEMPORARY VIEW") { + withTempView("myabcdview") { + val e = intercept[ParseException] { + sql("CREATE TEMPORARY VIEW myabcdview TBLPROPERTIES ('a' = 'b') AS SELECT * FROM jt") + } + assert(e.message.contains( + "Operation not allowed: TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW")) + } + } + test("correctly parse CREATE VIEW statement") { withView("testView") { sql( @@ -301,7 +312,6 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql( """CREATE TEMPORARY VIEW |testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') - |TBLPROPERTIES ('a' = 'b') |AS SELECT * FROM jt |""".stripMargin) checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala index f5c3b7816f5ea..94e22a414a628 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala @@ -198,9 +198,11 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite { // the size of data is 0. val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) + // Create at least one partition spec + val expectedPartitionSpecs = Seq(CoalescedPartitionSpec(0, 5)) checkEstimation( Array(bytesByPartitionId1, bytesByPartitionId2), - Seq.empty, targetSize, minNumPartitions) + expectedPartitionSpecs, targetSize, minNumPartitions) } @@ -248,16 +250,19 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite { val minNumPartitions = 2 { - // 1 shuffle: All bytes per partition are 0, no partition spec created. + // 1 shuffle: All bytes per partition are 0, 1 empty partition spec created. val bytesByPartitionId = Array[Long](0, 0, 0, 0, 0) - checkEstimation(Array(bytesByPartitionId), Seq.empty, targetSize) + val expectedPartitionSpecs = Seq(CoalescedPartitionSpec(0, 5)) + checkEstimation(Array(bytesByPartitionId), expectedPartitionSpecs, targetSize) } { - // 2 shuffles: All bytes per partition are 0, no partition spec created. + // 2 shuffles: All bytes per partition are 0, 1 empty partition spec created. val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) - checkEstimation(Array(bytesByPartitionId1, bytesByPartitionId2), Seq.empty, targetSize) + val expectedPartitionSpecs = Seq(CoalescedPartitionSpec(0, 5)) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionSpecs, targetSize) } { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala index 9107f8afa83d7..b4cb7e3fce3cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala @@ -37,9 +37,9 @@ class SparkPlannerSuite extends SharedSparkSession { case ReturnAnswer(child) => planned += 1 planLater(child) :: planLater(NeverPlanned) :: Nil - case Union(children) => + case u: Union => planned += 1 - UnionExec(children.map(planLater)) :: planLater(NeverPlanned) :: Nil + UnionExec(u.children.map(planLater)) :: planLater(NeverPlanned) :: Nil case LocalRelation(output, data, _) => planned += 1 LocalTableScanExec(output, data) :: planLater(NeverPlanned) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 1991f139e48c5..62712cf72eb59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.execution +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} @@ -25,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} -import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} /** @@ -61,6 +64,82 @@ class SparkSqlParserSuite extends AnalysisTest { private def intercept(sqlCommand: String, messages: String*): Unit = interceptParseException(parser.parsePlan)(sqlCommand, messages: _*) + test("Checks if SET/RESET can parse all the configurations") { + // Force to build static SQL configurations + StaticSQLConf + ConfigEntry.knownConfigs.values.asScala.foreach { config => + assertEqual(s"SET ${config.key}", SetCommand(Some(config.key -> None))) + if (config.defaultValue.isDefined && config.defaultValueString != null) { + assertEqual(s"SET ${config.key}=${config.defaultValueString}", + SetCommand(Some(config.key -> Some(config.defaultValueString)))) + } + assertEqual(s"RESET ${config.key}", ResetCommand(Some(config.key))) + } + } + + test("Report Error for invalid usage of SET command") { + assertEqual("SET", SetCommand(None)) + assertEqual("SET -v", SetCommand(Some("-v", None))) + assertEqual("SET spark.sql.key", SetCommand(Some("spark.sql.key" -> None))) + assertEqual("SET spark.sql.key ", SetCommand(Some("spark.sql.key" -> None))) + assertEqual("SET spark:sql:key=false", SetCommand(Some("spark:sql:key" -> Some("false")))) + assertEqual("SET spark:sql:key=", SetCommand(Some("spark:sql:key" -> Some("")))) + assertEqual("SET spark:sql:key= ", SetCommand(Some("spark:sql:key" -> Some("")))) + assertEqual("SET spark:sql:key=-1 ", SetCommand(Some("spark:sql:key" -> Some("-1")))) + assertEqual("SET spark:sql:key = -1", SetCommand(Some("spark:sql:key" -> Some("-1")))) + assertEqual("SET 1.2.key=value", SetCommand(Some("1.2.key" -> Some("value")))) + assertEqual("SET spark.sql.3=4", SetCommand(Some("spark.sql.3" -> Some("4")))) + assertEqual("SET 1:2:key=value", SetCommand(Some("1:2:key" -> Some("value")))) + assertEqual("SET spark:sql:3=4", SetCommand(Some("spark:sql:3" -> Some("4")))) + assertEqual("SET 5=6", SetCommand(Some("5" -> Some("6")))) + assertEqual("SET spark:sql:key = va l u e ", + SetCommand(Some("spark:sql:key" -> Some("va l u e")))) + assertEqual("SET `spark.sql. key`=value", + SetCommand(Some("spark.sql. key" -> Some("value")))) + assertEqual("SET `spark.sql. key`= v a lu e ", + SetCommand(Some("spark.sql. key" -> Some("v a lu e")))) + assertEqual("SET `spark.sql. key`= -1", + SetCommand(Some("spark.sql. key" -> Some("-1")))) + + val expectedErrMsg = "Expected format is 'SET', 'SET key', or " + + "'SET key=value'. If you want to include special characters in key, " + + "please use quotes, e.g., SET `ke y`=value." + intercept("SET spark.sql.key value", expectedErrMsg) + intercept("SET spark.sql.key 'value'", expectedErrMsg) + intercept("SET spark.sql.key \"value\" ", expectedErrMsg) + intercept("SET spark.sql.key value1 value2", expectedErrMsg) + intercept("SET spark. sql.key=value", expectedErrMsg) + intercept("SET spark :sql:key=value", expectedErrMsg) + intercept("SET spark . sql.key=value", expectedErrMsg) + intercept("SET spark.sql. key=value", expectedErrMsg) + intercept("SET spark.sql :key=value", expectedErrMsg) + intercept("SET spark.sql . key=value", expectedErrMsg) + } + + test("Report Error for invalid usage of RESET command") { + assertEqual("RESET", ResetCommand(None)) + assertEqual("RESET spark.sql.key", ResetCommand(Some("spark.sql.key"))) + assertEqual("RESET spark.sql.key ", ResetCommand(Some("spark.sql.key"))) + assertEqual("RESET 1.2.key ", ResetCommand(Some("1.2.key"))) + assertEqual("RESET spark.sql.3", ResetCommand(Some("spark.sql.3"))) + assertEqual("RESET 1:2:key ", ResetCommand(Some("1:2:key"))) + assertEqual("RESET spark:sql:3", ResetCommand(Some("spark:sql:3"))) + assertEqual("RESET `spark.sql. key`", ResetCommand(Some("spark.sql. key"))) + + val expectedErrMsg = "Expected format is 'RESET' or 'RESET key'. " + + "If you want to include special characters in key, " + + "please use quotes, e.g., RESET `ke y`." + intercept("RESET spark.sql.key1 key2", expectedErrMsg) + intercept("RESET spark. sql.key1 key2", expectedErrMsg) + intercept("RESET spark.sql.key1 key2 key3", expectedErrMsg) + intercept("RESET spark: sql:key", expectedErrMsg) + intercept("RESET spark .sql.key", expectedErrMsg) + intercept("RESET spark : sql:key", expectedErrMsg) + intercept("RESET spark.sql: key", expectedErrMsg) + intercept("RESET spark.sql .key", expectedErrMsg) + intercept("RESET spark.sql : key", expectedErrMsg) + } + test("refresh resource") { assertEqual("REFRESH prefix_path", RefreshResource("prefix_path")) assertEqual("REFRESH /", RefreshResource("/")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index ef81f1b788496..97e5c1148c244 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -24,7 +24,8 @@ import scala.util.{Random, Try} import scala.util.control.NonFatal import org.mockito.Mockito._ -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala index c5a01de911962..ee5e0e09fca21 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.sql.execution -import org.scalatest.{Assertions, BeforeAndAfterEach, Matchers} -import org.scalatest.concurrent.TimeLimits +import org.scalatest.{Assertions, BeforeAndAfterEach} +import org.scalatest.matchers.must.Matchers import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkFunSuite, TestUtils} import org.apache.spark.deploy.SparkSubmitSuite import org.apache.spark.internal.Logging -import org.apache.spark.sql.{LocalSparkSession, QueryTest, Row, SparkSession} +import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.functions.{array, col, count, lit} import org.apache.spark.sql.types.IntegerType import org.apache.spark.unsafe.Platform diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index f7396ee2a89c8..fe40d7dce344d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -22,8 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAnd import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec -import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -71,6 +70,31 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) } + test("ShuffledHashJoin should be included in WholeStageCodegen") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + val df1 = spark.range(5).select($"id".as("k1")) + val df2 = spark.range(15).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) + + // test one shuffled hash join + val oneJoinDF = df1.join(df2, $"k1" === $"k2") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) + + // test two shuffled hash joins + val twoJoinsDF = df1.join(df2, $"k1" === $"k2").join(df3, $"k1" === $"k3") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) + } + } + test("Sort should be included in WholeStageCodegen") { val df = spark.range(3, 0, -1).toDF().sort(col("id")) val plan = df.queryExecution.executedPlan @@ -342,7 +366,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(baseTable, "idx") assert(distinctWithId.queryExecution.executedPlan.collectFirst { case WholeStageCodegenExec( - ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _, _))) => true }.isDefined) checkAnswer(distinctWithId, Seq(Row(1, 0), Row(1, 0))) @@ -353,7 +377,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(baseTable, "idx") assert(groupByWithId.queryExecution.executedPlan.collectFirst { case WholeStageCodegenExec( - ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _, _))) => true }.isDefined) checkAnswer(groupByWithId, Seq(Row(1, 2, 0), Row(1, 2, 0))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index c696d3f648ed1..7fdcbd0d089cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -68,7 +68,7 @@ class AdaptiveQueryExecSuite val result = dfAdaptive.collect() withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { val df = sql(query) - QueryTest.sameRows(result.toSeq, df.collect().toSeq) + checkAnswer(df, result) } val planAfter = dfAdaptive.queryExecution.executedPlan assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) @@ -213,7 +213,7 @@ class AdaptiveQueryExecSuite } } - test("Empty stage coalesced to 0-partition RDD") { + test("Empty stage coalesced to 1-partition RDD") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true") { @@ -227,8 +227,8 @@ class AdaptiveQueryExecSuite val coalescedReaders = collect(plan) { case r: CustomShuffleReaderExec => r } - assert(coalescedReaders.length == 2) - coalescedReaders.foreach(r => assert(r.partitionSpecs.isEmpty)) + assert(coalescedReaders.length == 3) + coalescedReaders.foreach(r => assert(r.partitionSpecs.length == 1)) } withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { @@ -239,8 +239,8 @@ class AdaptiveQueryExecSuite val coalescedReaders = collect(plan) { case r: CustomShuffleReaderExec => r } - assert(coalescedReaders.length == 2, s"$plan") - coalescedReaders.foreach(r => assert(r.partitionSpecs.isEmpty)) + assert(coalescedReaders.length == 3, s"$plan") + coalescedReaders.foreach(r => assert(r.isLocalReader || r.partitionSpecs.length == 1)) } } } @@ -557,7 +557,8 @@ class AdaptiveQueryExecSuite withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT.key -> "0") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM testData " + "join testData2 t2 ON key = t2.a " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoinSuite.scala similarity index 55% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoinSuite.scala index 6a70045c55e64..89aee37a4246f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInSortMergeJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/bucketing/CoalesceBucketsInJoinSuite.scala @@ -19,17 +19,21 @@ package org.apache.spark.sql.execution.bucketing import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.execution.{BinaryExecNode, FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, PartitionSpec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types.{IntegerType, StructType} -class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkSession { +class CoalesceBucketsInJoinSuite extends SQLTestUtils with SharedSparkSession { + private val SORT_MERGE_JOIN = "sortMergeJoin" + private val SHUFFLED_HASH_JOIN = "shuffledHashJoin" + private val BROADCAST_HASH_JOIN = "broadcastHashJoin" + case class RelationSetting( cols: Seq[Attribute], numBuckets: Int, @@ -47,11 +51,16 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys: Seq[Attribute], leftRelation: RelationSetting, rightRelation: RelationSetting, - isSortMergeJoin: Boolean) + joinOperator: String, + shjBuildSide: Option[BuildSide]) object JoinSetting { - def apply(l: RelationSetting, r: RelationSetting, isSortMergeJoin: Boolean): JoinSetting = { - JoinSetting(l.cols, r.cols, l, r, isSortMergeJoin) + def apply( + l: RelationSetting, + r: RelationSetting, + joinOperator: String = SORT_MERGE_JOIN, + shjBuildSide: Option[BuildSide] = None): JoinSetting = { + JoinSetting(l.cols, r.cols, l, r, joinOperator, shjBuildSide) } } @@ -73,17 +82,24 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS leftRelation = setting.rightRelation, rightRelation = setting.leftRelation) - Seq(setting, swappedSetting).foreach { case s => + val settings = if (setting.joinOperator != SHUFFLED_HASH_JOIN) { + Seq(setting, swappedSetting) + } else { + Seq(setting) + } + settings.foreach { s => val lScan = newFileSourceScanExec(s.leftRelation) val rScan = newFileSourceScanExec(s.rightRelation) - val join = if (s.isSortMergeJoin) { + val join = if (s.joinOperator == SORT_MERGE_JOIN) { SortMergeJoinExec(s.leftKeys, s.rightKeys, Inner, None, lScan, rScan) + } else if (s.joinOperator == SHUFFLED_HASH_JOIN) { + ShuffledHashJoinExec(s.leftKeys, s.rightKeys, Inner, s.shjBuildSide.get, None, lScan, rScan) } else { BroadcastHashJoinExec( s.leftKeys, s.rightKeys, Inner, BuildLeft, None, lScan, rScan) } - val plan = CoalesceBucketsInSortMergeJoin(spark.sessionState.conf)(join) + val plan = CoalesceBucketsInJoin(spark.sessionState.conf)(join) def verify(expected: Option[Int], subPlan: SparkPlan): Unit = { val coalesced = subPlan.collect { @@ -91,7 +107,7 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS f.optionalNumCoalescedBuckets.get } if (expected.isDefined) { - assert(coalesced.size == 1 && coalesced(0) == expected.get) + assert(coalesced.size == 1 && coalesced.head == expected.get) } else { assert(coalesced.isEmpty) } @@ -103,46 +119,73 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS } test("bucket coalescing - basic") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, Some(4)), joinOperator = SORT_MERGE_JOIN)) run(JoinSetting( - RelationSetting(4, None), RelationSetting(8, Some(4)), isSortMergeJoin = true)) + RelationSetting(4, None), RelationSetting(8, Some(4)), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { - run(JoinSetting(RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = true)) + + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } - test("bucket coalescing should work only for sort merge join") { + test("bucket coalescing should work only for sort merge join and shuffled hash join") { Seq(true, false).foreach { enabled => - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> enabled.toString) { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> enabled.toString) { run(JoinSetting( - RelationSetting(4, None), RelationSetting(8, None), isSortMergeJoin = false)) + RelationSetting(4, None), RelationSetting(8, None), joinOperator = BROADCAST_HASH_JOIN)) } } } + test("bucket coalescing shouldn't be applied to shuffled hash join build side") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildRight))) + } + } + test("bucket coalescing shouldn't be applied when the number of buckets are the same") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { - run(JoinSetting(RelationSetting(8, None), RelationSetting(8, None), isSortMergeJoin = true)) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(8, None), RelationSetting(8, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(8, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } test("number of bucket is not divisible by other number of bucket") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { - run(JoinSetting(RelationSetting(3, None), RelationSetting(8, None), isSortMergeJoin = true)) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { + run(JoinSetting( + RelationSetting(3, None), RelationSetting(8, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(3, None), RelationSetting(8, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } test("the ratio of the number of buckets is greater than max allowed") { - withSQLConf( - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { - run(JoinSetting(RelationSetting(4, None), RelationSetting(16, None), isSortMergeJoin = true)) + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO.key -> "2") { + run(JoinSetting( + RelationSetting(4, None), RelationSetting(16, None), joinOperator = SORT_MERGE_JOIN)) + run(JoinSetting( + RelationSetting(4, None), RelationSetting(16, None), joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) } } test("join keys should match with output partitioning") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { val lCols = Seq( AttributeReference("l1", IntegerType)(), AttributeReference("l2", IntegerType)()) @@ -160,7 +203,16 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys = Seq(rCols.head), leftRelation = lRel, rightRelation = rRel, - isSortMergeJoin = true)) + joinOperator = SORT_MERGE_JOIN, + shjBuildSide = None)) + + run(JoinSetting( + leftKeys = Seq(lCols.head), + rightKeys = Seq(rCols.head), + leftRelation = lRel, + rightRelation = rRel, + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) // The following should not be coalesced because join keys do not match with output // partitioning (more expressions). @@ -169,7 +221,16 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys = rCols :+ AttributeReference("r3", IntegerType)(), leftRelation = lRel, rightRelation = rRel, - isSortMergeJoin = true)) + joinOperator = SORT_MERGE_JOIN, + shjBuildSide = None)) + + run(JoinSetting( + leftKeys = lCols :+ AttributeReference("l3", IntegerType)(), + rightKeys = rCols :+ AttributeReference("r3", IntegerType)(), + leftRelation = lRel, + rightRelation = rRel, + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) // The following will be coalesced since ordering should not matter because it will be // adjusted in `EnsureRequirements`. @@ -178,7 +239,24 @@ class CoalesceBucketsInSortMergeJoinSuite extends SQLTestUtils with SharedSparkS rightKeys = rCols.reverse, leftRelation = lRel, rightRelation = RelationSetting(rCols, 8, Some(4)), - isSortMergeJoin = true)) + joinOperator = SORT_MERGE_JOIN, + shjBuildSide = None)) + + run(JoinSetting( + leftKeys = lCols.reverse, + rightKeys = rCols.reverse, + leftRelation = lRel, + rightRelation = RelationSetting(rCols, 8, Some(4)), + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildLeft))) + + run(JoinSetting( + leftKeys = rCols.reverse, + rightKeys = lCols.reverse, + leftRelation = RelationSetting(rCols, 8, Some(4)), + rightRelation = lRel, + joinOperator = SHUFFLED_HASH_JOIN, + shjBuildSide = Some(BuildRight))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala new file mode 100644 index 0000000000000..099a1aa996c11 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/CachedBatchSerializerSuite.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar + +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} +import org.apache.spark.sql.execution.columnar.InMemoryRelation.clearSerializer +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.storage.StorageLevel + +case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch { + override def numRows: Int = data.length + override def sizeInBytes: Long = 4 * data.length +} + +/** + * Very simple serializer that only supports a single int column, but does support columnar. + */ +class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer { + override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true + + override def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + if (schema.length != 1 || schema.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing $schema") + } + input.map { cb => + val column = cb.column(0) + val data = column.getInts(0, cb.numRows()) + SingleIntCachedBatch(data) + } + } + + override def supportsColumnarOutput(schema: StructType): Boolean = true + override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = + Some(attributes.map(_ => classOf[OnHeapColumnVector].getName)) + + override def convertCachedBatchToColumnarBatch( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[ColumnarBatch] = { + if (selectedAttributes.isEmpty) { + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + new ColumnarBatch(new Array[ColumnVector](0), single.numRows) + } + } else { + if (selectedAttributes.length > 1 || + selectedAttributes.head.dataType != IntegerType) { + throw new IllegalArgumentException("Only a single column of non-nullable ints works. " + + s"This is for testing") + } + input.map { cached => + val single = cached.asInstanceOf[SingleIntCachedBatch] + val cv = OnHeapColumnVector.allocateColumns(single.numRows, selectedAttributes.toStructType) + val data = single.data + cv(0).putInts(0, data.length, data, 0) + new ColumnarBatch(cv.toArray, single.numRows) + } + } + } + + override def convertCachedBatchToInternalRow( + input: RDD[CachedBatch], + cacheAttributes: Seq[Attribute], + selectedAttributes: Seq[Attribute], + conf: SQLConf): RDD[InternalRow] = { + throw new IllegalStateException("This does not work. This is only for testing") + } + + override def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + def ret(index: Int, cb: Iterator[CachedBatch]): Iterator[CachedBatch] = cb + ret + } +} + +class CachedBatchSerializerSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + override protected def sparkConf: SparkConf = { + super.sparkConf.set( + StaticSQLConf.SPARK_CACHE_SERIALIZER.key, + classOf[TestSingleIntColumnarCachedBatchSerializer].getName) + } + + protected override def beforeAll(): Unit = { + super.beforeAll() + clearSerializer() + } + + protected override def afterAll(): Unit = { + clearSerializer() + super.afterAll() + } + + test("Columnar Cache Plugin") { + withTempPath { workDir => + val workDirPath = workDir.getAbsolutePath + val input = Seq(100, 200, 300).toDF("count") + input.write.parquet(workDirPath) + val data = spark.read.parquet(workDirPath) + data.cache() + assert(data.count() == 3) + checkAnswer(data, Row(100) :: Row(200) :: Row(300) :: Nil) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 18f29f7b90ad5..b8f73f4563ef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -20,18 +20,32 @@ package org.apache.spark.sql.execution.columnar import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec} +import org.apache.spark.sql.columnar.CachedBatch +import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ -import org.apache.spark.util.Utils + +class TestCachedBatchSerializer( + useCompression: Boolean, + batchSize: Int) extends DefaultCachedBatchSerializer { + + override def convertInternalRowToCachedBatch(input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] = { + convertForCacheInternal(input, schema, batchSize, useCompression) + } +} class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -42,12 +56,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { data.createOrReplaceTempView(s"testData$dataType") val storageLevel = MEMORY_ONLY val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None, - data.logicalPlan) + val inMemoryRelation = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + storageLevel, plan, None, data.logicalPlan) assert(inMemoryRelation.cacheBuilder.cachedColumnBuffers.getStorageLevel == storageLevel) inMemoryRelation.cacheBuilder.cachedColumnBuffers.collect().head match { - case _: CachedBatch => + case _: DefaultCachedBatch => case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") } checkAnswer(inMemoryRelation, data.collect().toSeq) @@ -119,8 +133,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("simple columnar query") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) } @@ -140,8 +154,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("projection") { val logicalPlan = testData.select('value, 'key).logicalPlan val plan = spark.sessionState.executePlan(logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, logicalPlan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -157,8 +171,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None, - testData.logicalPlan) + val scan = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5), + MEMORY_ONLY, plan, None, testData.logicalPlan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) @@ -336,7 +350,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-17549: cached table size should be correctly calculated") { val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -349,7 +364,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("cached row count should be calculated") { val data = spark.range(6).toDF val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan - val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + val cached = InMemoryRelation(new TestCachedBatchSerializer(true, 5), + MEMORY_ONLY, plan, None, data.logicalPlan) // Materialize the data. val expectedAnswer = data.collect() @@ -474,12 +490,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { test("SPARK-22249: buildFilter should not throw exception when In contains an empty list") { val attribute = AttributeReference("a", IntegerType)() - val localTableScanExec = LocalTableScanExec(Seq(attribute), Nil) - val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY, localTableScanExec, None, - LocalRelation(Seq(attribute), Nil)) - val tableScanExec = InMemoryTableScanExec(Seq(attribute), - Seq(In(attribute, Nil)), testRelation) - assert(tableScanExec.partitionFilters.isEmpty) + val testSerializer = new TestCachedBatchSerializer(false, 1) + testSerializer.buildFilter(Seq(In(attribute, Nil)), Seq(attribute)) } testWithWholeStageCodegenOnAndOff("SPARK-22348: table cache " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e4709e469dca3..faafcb721008f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -28,8 +28,8 @@ import org.apache.spark.{SparkException, SparkFiles} import org.apache.spark.internal.config import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} -import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchFunctionException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER @@ -3030,6 +3030,49 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } } + + test("REFRESH FUNCTION") { + val msg = intercept[AnalysisException] { + sql("REFRESH FUNCTION md5") + }.getMessage + assert(msg.contains("Cannot refresh builtin function")) + + withUserDefinedFunction("func1" -> true) { + sql("CREATE TEMPORARY FUNCTION func1 AS 'test.org.apache.spark.sql.MyDoubleAvg'") + val msg = intercept[AnalysisException] { + sql("REFRESH FUNCTION func1") + }.getMessage + assert(msg.contains("Cannot refresh temporary function")) + } + + withUserDefinedFunction("func1" -> false) { + intercept[NoSuchFunctionException] { + sql("REFRESH FUNCTION func1") + } + + val func = FunctionIdentifier("func1", Some("default")) + sql("CREATE FUNCTION func1 AS 'test.org.apache.spark.sql.MyDoubleAvg'") + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + sql("REFRESH FUNCTION func1") + assert(spark.sessionState.catalog.isRegisteredFunction(func)) + + spark.sessionState.catalog.externalCatalog.dropFunction("default", "func1") + assert(spark.sessionState.catalog.isRegisteredFunction(func)) + intercept[NoSuchFunctionException] { + sql("REFRESH FUNCTION func1") + } + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + + val function = CatalogFunction(func, "test.non.exists.udf", Seq.empty) + spark.sessionState.catalog.createFunction(function, false) + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + val err = intercept[AnalysisException] { + sql("REFRESH FUNCTION func1") + }.getMessage + assert(err.contains("Can not load class")) + assert(!spark.sessionState.catalog.isRegisteredFunction(func)) + } + } } object FakeLocalFsFileSystem { 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 ea15f1891b006..02be8c9221704 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 @@ -21,6 +21,7 @@ import java.io.{File, FileNotFoundException} import java.net.URI import scala.collection.mutable +import scala.concurrent.duration._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path, RawLocalFileSystem, RemoteIterator} @@ -33,7 +34,7 @@ import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.functions.col -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.KnownSizeEstimation @@ -366,13 +367,15 @@ class FileIndexSuite extends SharedSparkSession { val wrongBasePath = new File(dir, "unknown") // basePath must be a directory wrongBasePath.mkdir() - val parameters = Map("basePath" -> wrongBasePath.getCanonicalPath) - val fileIndex = new InMemoryFileIndex(spark, Seq(path), parameters, None) - val msg = intercept[IllegalArgumentException] { - // trigger inferPartitioning() - fileIndex.partitionSpec() - }.getMessage - assert(msg === s"Wrong basePath ${wrongBasePath.getCanonicalPath} for the root path: $path") + withClue("SPARK-32368: 'basePath' can be case insensitive") { + val parameters = Map("bAsepAtH" -> wrongBasePath.getCanonicalPath) + val fileIndex = new InMemoryFileIndex(spark, Seq(path), parameters, None) + val msg = intercept[IllegalArgumentException] { + // trigger inferPartitioning() + fileIndex.partitionSpec() + }.getMessage + assert(msg === s"Wrong basePath ${wrongBasePath.getCanonicalPath} for the root path: $path") + } } } @@ -488,6 +491,31 @@ class FileIndexSuite extends SharedSparkSession { val fileIndex = new TestInMemoryFileIndex(spark, path) assert(fileIndex.leafFileStatuses.toSeq == statuses) } + + test("expire FileStatusCache if TTL is configured") { + val previousValue = SQLConf.get.getConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS) + try { + // using 'SQLConf.get.setConf' instead of 'withSQLConf' to set a static config at runtime + SQLConf.get.setConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS, 1L) + + val path = new Path("/dummy_tmp", "abc") + val files = (1 to 3).map(_ => new FileStatus()) + + FileStatusCache.resetForTesting() + val fileStatusCache = FileStatusCache.getOrCreate(spark) + fileStatusCache.putLeafFiles(path, files.toArray) + + // Exactly 3 files are cached. + assert(fileStatusCache.getLeafFiles(path).get.length === 3) + // Wait until the cache expiration. + eventually(timeout(3.seconds)) { + // And the cache is gone. + assert(fileStatusCache.getLeafFiles(path).isEmpty === true) + } + } finally { + SQLConf.get.setConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS, previousValue) + } + } } object DeletionRaceFileSystem { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index d51eafa5a8aed..2b5cb27d59ad9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -460,6 +460,69 @@ abstract class SchemaPruningSuite checkAnswer(query4, Row(2, null) :: Row(2, 4) :: Nil) } + testSchemaPruning("select nested field in window function") { + val windowSql = + """ + |with contact_rank as ( + | select row_number() over (partition by address order by id desc) as rank, + | contacts.* + | from contacts + |) + |select name.first, rank from contact_rank + |where name.first = 'Jane' AND rank = 1 + |""".stripMargin + val query = sql(windowSql) + checkScan(query, "struct,address:string>") + checkAnswer(query, Row("Jane", 1) :: Nil) + } + + testSchemaPruning("select nested field in window function and then order by") { + val windowSql = + """ + |with contact_rank as ( + | select row_number() over (partition by address order by id desc) as rank, + | contacts.* + | from contacts + | order by name.last, name.first + |) + |select name.first, rank from contact_rank + |""".stripMargin + val query = sql(windowSql) + checkScan(query, "struct,address:string>") + checkAnswer(query, + Row("Jane", 1) :: + Row("John", 1) :: + Row("Janet", 1) :: + Row("Jim", 1) :: Nil) + } + + testSchemaPruning("select nested field in Sort") { + val query1 = sql("select name.first, name.last from contacts order by name.first, name.last") + checkScan(query1, "struct>") + checkAnswer(query1, + Row("Jane", "Doe") :: + Row("Janet", "Jones") :: + Row("Jim", "Jones") :: + Row("John", "Doe") :: Nil) + + withTempView("tmp_contacts") { + // Create a repartitioned view because `SORT BY` is a local sort + sql("select * from contacts").repartition(1).createOrReplaceTempView("tmp_contacts") + val sortBySql = + """ + |select name.first, name.last from tmp_contacts + |sort by name.first, name.last + |""".stripMargin + val query2 = sql(sortBySql) + checkScan(query2, "struct>") + checkAnswer(query2, + Row("Jane", "Doe") :: + Row("Janet", "Jones") :: + Row("Jim", "Jones") :: + Row("John", "Doe") :: Nil) + } + } + testSchemaPruning("select nested field in Expand") { import org.apache.spark.sql.catalyst.dsl.expressions._ 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 fcb7bdc25f08f..db6f45247d130 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 @@ -2353,6 +2353,36 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa assert(df.schema.last == StructField("col_mixed_types", StringType, true)) } } + + test("case sensitivity of filters references") { + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + withTempPath { path => + Seq( + """aaa,BBB""", + """0,1""", + """2,3""").toDF().repartition(1).write.text(path.getCanonicalPath) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val readback = spark.read.schema("aaa integer, BBB integer") + .option("header", true) + .csv(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(2, 3), Row(0, 1))) + checkAnswer(readback.filter($"AAA" === 2 && $"bbb" === 3), Seq(Row(2, 3))) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val readback = spark.read.schema("aaa integer, BBB integer") + .option("header", true) + .csv(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(2, 3), Row(0, 1))) + val errorMsg = intercept[AnalysisException] { + readback.filter($"AAA" === 2 && $"bbb" === 3).collect() + }.getMessage + assert(errorMsg.contains("cannot resolve '`AAA`'")) + } + } + } + } + } } class CSVv1Suite extends CSVSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala index 0dbd6b5754afb..9ff35c0946cc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala @@ -20,9 +20,10 @@ import java.io.File import java.time.{Instant, LocalDate} import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.{Column, Dataset, Row} import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -495,6 +496,45 @@ object JsonBenchmark extends SqlBasedBenchmark { } } + private def filtersPushdownBenchmark(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("Filters pushdown", rowsNum, output = output) + val colsNum = 100 + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", TimestampType)) + val schema = StructType(StructField("key", IntegerType) +: fields) + def columns(): Seq[Column] = { + val ts = Seq.tabulate(colsNum) { i => + lit(Instant.ofEpochSecond(i * 12345678)).as(s"col$i") + } + ($"id" % 1000).as("key") +: ts + } + withTempPath { path => + spark.range(rowsNum).select(columns(): _*).write.json(path.getAbsolutePath) + def readback = { + spark.read.schema(schema).json(path.getAbsolutePath) + } + + benchmark.addCase("w/o filters", numIters) { _ => + readback.noop() + } + + def withFilter(configEnabled: Boolean): Unit = { + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) { + readback.filter($"key" === 0).noop() + } + } + + benchmark.addCase("pushdown disabled", numIters) { _ => + withFilter(configEnabled = false) + } + + benchmark.addCase("w/ filters", numIters) { _ => + withFilter(configEnabled = true) + } + + benchmark.run() + } + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val numIters = 3 runBenchmark("Benchmark for performance of JSON parsing") { @@ -508,6 +548,9 @@ object JsonBenchmark extends SqlBasedBenchmark { jsonInDS(50 * 1000 * 1000, numIters) jsonInFile(50 * 1000 * 1000, numIters) datetimeBenchmark(rowsNum = 10 * 1000 * 1000, numIters) + // Benchmark pushdown filters that refer to top-level columns. + // TODO (SPARK-32325): Add benchmarks for filters with nested column attributes. + filtersPushdownBenchmark(rowsNum = 100 * 1000, numIters) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index 7592809d7c85b..d27b5c4737a11 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.datasources.json -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.json.JSONOptions import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{DoubleType, StringType, StructType} /** * Test cases for various [[JSONOptions]]. @@ -102,21 +103,32 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession { assert(df.first().getLong(0) == 18) } - // The following two tests are not really working - need to look into Jackson's - // JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS. - ignore("allowNonNumericNumbers off") { + test("allowNonNumericNumbers off") { val str = """{"age": NaN}""" - val df = spark.read.json(Seq(str).toDS()) + val df = spark.read.option("allowNonNumericNumbers", false).json(Seq(str).toDS()) - assert(df.schema.head.name == "_corrupt_record") + assert(df.schema === new StructType().add("_corrupt_record", StringType)) + checkAnswer(df, Row(str)) } - ignore("allowNonNumericNumbers on") { - val str = """{"age": NaN}""" - val df = spark.read.option("allowNonNumericNumbers", "true").json(Seq(str).toDS()) - - assert(df.schema.head.name == "age") - assert(df.first().getDouble(0).isNaN) + test("allowNonNumericNumbers on") { + val str = """{"c0":NaN, "c1":+INF, "c2":+Infinity, "c3":Infinity, "c4":-INF, "c5":-Infinity}""" + val df = spark.read.option("allowNonNumericNumbers", true).json(Seq(str).toDS()) + + assert(df.schema === + new StructType() + .add("c0", "double") + .add("c1", "double") + .add("c2", "double") + .add("c3", "double") + .add("c4", "double") + .add("c5", "double")) + checkAnswer( + df, + Row( + Double.NaN, + Double.PositiveInfinity, Double.PositiveInfinity, Double.PositiveInfinity, + Double.NegativeInfinity, Double.NegativeInfinity)) } test("allowBackslashEscapingAnyCharacter off") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index de01099f2db55..2a98cba663a1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -35,12 +35,14 @@ import org.apache.spark.sql.{functions => F, _} import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.ExternalRDD -import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, NoopCache} +import org.apache.spark.sql.execution.datasources.v2.json.JsonScanBuilder import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.sql.types.StructType.fromDDL import org.apache.spark.sql.types.TestUDT.{MyDenseVector, MyDenseVectorUDT} +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils class TestFileFilter extends PathFilter { @@ -2688,6 +2690,122 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson checkAnswer(json, Row(null)) } } + + test("filters push down") { + withTempPath { path => + val t = "2019-12-17 00:01:02" + Seq( + """{"c0": "abc", "c1": {"c2": 1, "c3": "2019-11-14 20:35:30"}}""", + s"""{"c0": "def", "c1": {"c2": 2, "c3": "$t"}}""", + s"""{"c0": "defa", "c1": {"c2": 3, "c3": "$t"}}""", + s"""{"c0": "define", "c1": {"c2": 2, "c3": "$t"}}""").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + Seq("PERMISSIVE", "DROPMALFORMED", "FAILFAST").foreach { mode => + val readback = spark.read + .option("mode", mode) + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .schema("c0 string, c1 struct") + .json(path.getAbsolutePath) + .where($"c1.c2" === 2 && $"c0".startsWith("def")) + .select($"c1.c3") + assert(readback.count() === 2) + checkAnswer(readback, Seq(Row(Timestamp.valueOf(t)), Row(Timestamp.valueOf(t)))) + } + } + } + } + } + + test("apply filters to malformed rows") { + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { path => + Seq( + "{}", + """{"invalid": 0}""", + """{"i":}""", + """{"i": 0}""", + """{"i": 1, "t": "2020-01-28 01:00:00"}""", + """{"t": "2020-01-28 02:00:00"}""", + """{"i": "abc", "t": "2020-01-28 03:00:00"}""", + """{"i": 2, "t": "2020-01-28 04:00:00", "d": 3.14}""").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + val schema = "i INTEGER, t TIMESTAMP" + val readback = spark.read + .schema(schema) + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .json(path.getAbsolutePath) + // readback: + // +----+-------------------+ + // |i |t | + // +----+-------------------+ + // |null|null | + // |null|null | + // |null|null | + // |0 |null | + // |1 |2020-01-28 01:00:00| + // |null|2020-01-28 02:00:00| + // |null|2020-01-28 03:00:00| + // |2 |2020-01-28 04:00:00| + // +----+-------------------+ + checkAnswer( + readback.where($"i".isNull && $"t".isNotNull), + Seq( + Row(null, Timestamp.valueOf("2020-01-28 02:00:00")), + Row(null, Timestamp.valueOf("2020-01-28 03:00:00")))) + checkAnswer( + readback.where($"i" >= 0 && $"t" > "2020-01-28 00:00:00"), + Seq( + Row(1, Timestamp.valueOf("2020-01-28 01:00:00")), + Row(2, Timestamp.valueOf("2020-01-28 04:00:00")))) + checkAnswer( + readback.where($"t".isNull).select($"i"), + Seq(Row(null), Row(null), Row(null), Row(0))) + } + } + } + + test("case sensitivity of filters references") { + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + withTempPath { path => + Seq( + """{"aaa": 0, "BBB": 1}""", + """{"AAA": 2, "bbb": 3}""").toDF().write.text(path.getCanonicalPath) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val readback = spark.read.schema("aaa integer, BBB integer") + .json(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(null, null), Row(0, 1))) + checkAnswer(readback.filter($"AAA" === 0 && $"bbb" === 1), Seq(Row(0, 1))) + checkAnswer(readback.filter($"AAA" === 2 && $"bbb" === 3), Seq()) + // Schema inferring + val errorMsg = intercept[AnalysisException] { + spark.read.json(path.getCanonicalPath).collect() + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the data schema")) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val readback = spark.read.schema("aaa integer, BBB integer") + .json(path.getCanonicalPath) + checkAnswer(readback, Seq(Row(null, null), Row(0, 1))) + val errorMsg = intercept[AnalysisException] { + readback.filter($"AAA" === 0 && $"bbb" === 1).collect() + }.getMessage + assert(errorMsg.contains("cannot resolve '`AAA`'")) + // Schema inferring + val readback2 = spark.read.json(path.getCanonicalPath) + checkAnswer( + readback2.filter($"AAA" === 2).select($"AAA", $"bbb"), + Seq(Row(2, 3))) + checkAnswer(readback2.filter($"aaa" === 2).select($"AAA", $"bbb"), Seq()) + } + } + } + } + } } class JsonV1Suite extends JsonSuite { @@ -2702,6 +2820,37 @@ class JsonV2Suite extends JsonSuite { super .sparkConf .set(SQLConf.USE_V1_SOURCE_LIST, "") + + test("get pushed filters") { + val attr = "col" + def getBuilder(path: String): JsonScanBuilder = { + val fileIndex = new InMemoryFileIndex( + spark, + Seq(new org.apache.hadoop.fs.Path(path, "file.json")), + Map.empty, + None, + NoopCache) + val schema = new StructType().add(attr, IntegerType) + val options = CaseInsensitiveStringMap.empty() + new JsonScanBuilder(spark, fileIndex, schema, schema, options) + } + val filters: Array[sources.Filter] = Array(sources.IsNotNull(attr)) + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushFilters(filters) === filters) + assert(scanBuilder.pushedFilters() === filters) + } + } + + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "false") { + withTempPath { file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushFilters(filters) === filters) + assert(scanBuilder.pushedFilters() === Array.empty[sources.Filter]) + } + } + } } class JsonLegacyTimeParserSuite extends JsonSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index a14f6416199a1..8c4eedfde76cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -165,7 +165,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS // the assumption on column stats, and also the end-to-end behavior. val hadoopConf = spark.sessionState.newHadoopConf() - val fs = FileSystem.get(hadoopConf) + val fs = new Path(tableDir.getAbsolutePath).getFileSystem(hadoopConf) val parts = fs.listStatus(new Path(tableDir.getAbsolutePath), new PathFilter { override def accept(path: Path): Boolean = !path.getName.startsWith("_") }) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index c399a011f9073..c3bcf86c1ed27 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.sql.execution.datasources.v2 +import java.net.URI import java.util import java.util.Collections import scala.collection.JavaConverters._ +import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -160,6 +162,36 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(catalog.tableExists(testIdent)) } + private def makeQualifiedPathWithWarehouse(path: String): URI = { + val p = new Path(spark.sessionState.conf.warehousePath, path) + val fs = p.getFileSystem(spark.sessionState.newHadoopConf()) + fs.makeQualified(p).toUri + + } + + test("createTable: location") { + val catalog = newCatalog() + val properties = new util.HashMap[String, String]() + assert(!catalog.tableExists(testIdent)) + + // default location + val t1 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t1.catalogTable.location === + spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) + catalog.dropTable(testIdent) + + // relative path + properties.put(TableCatalog.PROP_LOCATION, "relative/path") + val t2 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) + catalog.dropTable(testIdent) + + // absolute path + properties.put(TableCatalog.PROP_LOCATION, "/absolute/path") + val t3 = catalog.createTable(testIdent, schema, Array.empty, properties).asInstanceOf[V1Table] + assert(t3.catalogTable.location.toString === "file:/absolute/path") + } + test("tableExists") { val catalog = newCatalog() @@ -640,6 +672,26 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(exc.message.contains("not found")) } + test("alterTable: location") { + val catalog = newCatalog() + assert(!catalog.tableExists(testIdent)) + + // default location + val t1 = catalog.createTable(testIdent, schema, Array.empty, emptyProps).asInstanceOf[V1Table] + assert(t1.catalogTable.location === + spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) + + // relative path + val t2 = catalog.alterTable(testIdent, + TableChange.setProperty(TableCatalog.PROP_LOCATION, "relative/path")).asInstanceOf[V1Table] + assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) + + // absolute path + val t3 = catalog.alterTable(testIdent, + TableChange.setProperty(TableCatalog.PROP_LOCATION, "/absolute/path")).asInstanceOf[V1Table] + assert(t3.catalogTable.location.toString === "file:/absolute/path") + } + test("dropTable") { val catalog = newCatalog() @@ -812,11 +864,15 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { test("createNamespace: basic behavior") { val catalog = newCatalog() - val expectedPath = sqlContext.sessionState.catalog.getDefaultDBPath(testNs(0)).toString + + val sessionCatalog = sqlContext.sessionState.catalog + val expectedPath = + new Path(spark.sessionState.conf.warehousePath, + sessionCatalog.getDefaultDBPath(testNs(0)).toString).toString catalog.createNamespace(testNs, Map("property" -> "value").asJava) - assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) + assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri) assert(catalog.namespaceExists(testNs) === true) val metadata = catalog.loadNamespaceMetadata(testNs).asScala @@ -842,6 +898,23 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { catalog.dropNamespace(testNs) } + test("createNamespace: relative location") { + val catalog = newCatalog() + val expectedPath = + new Path(spark.sessionState.conf.warehousePath, "a/b/c").toString + + catalog.createNamespace(testNs, Map("location" -> "a/b/c").asJava) + + assert(expectedPath === spark.catalog.getDatabase(testNs(0)).locationUri) + + assert(catalog.namespaceExists(testNs) === true) + val metadata = catalog.loadNamespaceMetadata(testNs).asScala + checkMetadata(metadata, Map.empty) + assert(expectedPath === metadata("location")) + + catalog.dropNamespace(testNs) + } + test("createNamespace: fail if namespace already exists") { val catalog = newCatalog() @@ -954,16 +1027,23 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { test("alterNamespace: update namespace location") { val catalog = newCatalog() - val initialPath = sqlContext.sessionState.catalog.getDefaultDBPath(testNs(0)).toString - val newPath = "file:/tmp/db.db" + val initialPath = + new Path(spark.sessionState.conf.warehousePath, + spark.sessionState.catalog.getDefaultDBPath(testNs(0)).toString).toString + val newAbsoluteUri = "file:/tmp/db.db" catalog.createNamespace(testNs, emptyProps) + assert(initialPath === spark.catalog.getDatabase(testNs(0)).locationUri) + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newAbsoluteUri)) + assert(newAbsoluteUri === spark.catalog.getDatabase(testNs(0)).locationUri) - assert(initialPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) - - catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newPath)) + val newAbsolutePath = "/tmp/newAbsolutePath" + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", newAbsolutePath)) + assert("file:" + newAbsolutePath === spark.catalog.getDatabase(testNs(0)).locationUri) - assert(newPath === spark.catalog.getDatabase(testNs(0)).locationUri.toString) + val newRelativePath = new Path(spark.sessionState.conf.warehousePath, "relativeP").toString + catalog.alterNamespace(testNs, NamespaceChange.setProperty("location", "relativeP")) + assert(newRelativePath === spark.catalog.getDatabase(testNs(0)).locationUri) catalog.dropNamespace(testNs) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala new file mode 100644 index 0000000000000..b308934ba03c0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.jdbc + +import java.sql.{Connection, DriverManager} +import java.util.Properties + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { + + val tempDir = Utils.createTempDir() + val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" + var conn: java.sql.Connection = null + + override def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.catalog.h2", classOf[JDBCTableCatalog].getName) + .set("spark.sql.catalog.h2.url", url) + .set("spark.sql.catalog.h2.driver", "org.h2.Driver") + + private def withConnection[T](f: Connection => T): T = { + val conn = DriverManager.getConnection(url, new Properties()) + try { + f(conn) + } finally { + conn.close() + } + } + + override def beforeAll(): Unit = { + super.beforeAll() + Utils.classForName("org.h2.Driver") + withConnection { conn => + conn.prepareStatement("""CREATE SCHEMA "test"""").executeUpdate() + conn.prepareStatement( + """CREATE TABLE "test"."people" (name TEXT(32) NOT NULL, id INTEGER NOT NULL)""") + .executeUpdate() + } + } + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDir) + super.afterAll() + } + + test("show tables") { + checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + } + + test("drop a table and test whether the table exists") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."to_drop" (id INTEGER)""").executeUpdate() + } + checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "to_drop"), Row("test", "people"))) + sql("DROP TABLE h2.test.to_drop") + checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + } + + test("rename a table") { + withTable("h2.test.dst_table") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."src_table" (id INTEGER)""").executeUpdate() + } + checkAnswer( + sql("SHOW TABLES IN h2.test"), + Seq(Row("test", "src_table"), Row("test", "people"))) + sql("ALTER TABLE h2.test.src_table RENAME TO test.dst_table") + checkAnswer( + sql("SHOW TABLES IN h2.test"), + Seq(Row("test", "dst_table"), Row("test", "people"))) + } + } + + test("load a table") { + val t = spark.table("h2.test.people") + val expectedSchema = new StructType() + .add("NAME", StringType) + .add("ID", IntegerType) + assert(t.schema === expectedSchema) + } + + test("create a table") { + withTable("h2.test.new_table") { + // TODO (SPARK-32427): Omit USING in CREATE TABLE + sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") + checkAnswer( + sql("SHOW TABLES IN h2.test"), + Seq(Row("test", "people"), Row("test", "new_table"))) + } + } + + test("alter table ... add column") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (C1 INTEGER, C2 STRING)") + var t = spark.table("h2.test.alt_table") + var expectedSchema = new StructType() + .add("ID", IntegerType) + .add("C1", IntegerType) + .add("C2", StringType) + assert(t.schema === expectedSchema) + sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (C3 DOUBLE)") + t = spark.table("h2.test.alt_table") + expectedSchema = expectedSchema.add("C3", DoubleType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... rename column") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table RENAME COLUMN ID TO C") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("C", IntegerType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... drop column") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (C1 INTEGER, C2 INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table DROP COLUMN C1") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("C2", IntegerType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... update column type") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN id TYPE DOUBLE") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("ID", DoubleType) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... update column nullability") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER NOT NULL) USING _") + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID DROP NOT NULL") + val t = spark.table("h2.test.alt_table") + val expectedSchema = new StructType().add("ID", IntegerType, nullable = true) + assert(t.schema === expectedSchema) + } + } + + test("alter table ... update column comment not supported") { + withTable("h2.test.alt_table") { + sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + val thrown = intercept[java.sql.SQLFeatureNotSupportedException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID COMMENT 'test'") + } + assert(thrown.getMessage.contains("Unsupported TableChange")) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index e9ef7c1a0c540..d2c9322685d94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -69,8 +69,9 @@ class DebuggingSuite extends SharedSparkSession with DisableAdaptiveExecutionSui } val output = captured.toString() + val hashedModeString = "HashedRelationBroadcastMode(List(input[0, bigint, false]),false)" assert(output.replaceAll("\\[id=#\\d+\\]", "[id=#x]").contains( - """== BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#x] == + s"""== BroadcastExchange $hashedModeString, [id=#x] == |Tuples output: 0 | id LongType: {} |== WholeStageCodegen (1) == diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index f7d5a899df1c9..7ff945f5cbfb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -21,13 +21,15 @@ import scala.reflect.ClassTag import org.apache.spark.AccumulatorSuite import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BitwiseAnd, BitwiseOr, Cast, Expression, Literal, ShiftLeft} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.BROADCAST -import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection} +import org.apache.spark.sql.execution.{DummySparkPlan, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.exchange.EnsureRequirements +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -415,6 +417,216 @@ abstract class BroadcastJoinSuiteBase extends QueryTest with SQLTestUtils assert(e.getMessage.contains(s"Could not execute broadcast in $timeout secs.")) } } + + test("broadcast join where streamed side's output partitioning is HashPartitioning") { + withTable("t1", "t3") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val df1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1") + val df2 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i2", "j2") + val df3 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i3", "j3") + df1.write.format("parquet").bucketBy(8, "i1", "j1").saveAsTable("t1") + df3.write.format("parquet").bucketBy(8, "i3", "j3").saveAsTable("t3") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + // join1 is a broadcast join where df2 is broadcasted. Note that output partitioning on the + // streamed side (t1) is HashPartitioning (bucketed files). + val join1 = t1.join(df2, t1("i1") === df2("i2") && t1("j1") === df2("j2")) + val plan1 = join1.queryExecution.executedPlan + assert(collect(plan1) { case e: ShuffleExchangeExec => e }.isEmpty) + val broadcastJoins = collect(plan1) { case b: BroadcastHashJoinExec => b } + assert(broadcastJoins.size == 1) + assert(broadcastJoins(0).outputPartitioning.isInstanceOf[PartitioningCollection]) + val p = broadcastJoins(0).outputPartitioning.asInstanceOf[PartitioningCollection] + assert(p.partitionings.size == 4) + // Verify all the combinations of output partitioning. + Seq(Seq(t1("i1"), t1("j1")), + Seq(t1("i1"), df2("j2")), + Seq(df2("i2"), t1("j1")), + Seq(df2("i2"), df2("j2"))).foreach { expected => + val expectedExpressions = expected.map(_.expr) + assert(p.partitionings.exists { + case h: HashPartitioning => expressionsEqual(h.expressions, expectedExpressions) + }) + } + + // Join on the column from the broadcasted side (i2, j2) and make sure output partitioning + // is maintained by checking no shuffle exchange is introduced. + val join2 = join1.join(t3, join1("i2") === t3("i3") && join1("j2") === t3("j3")) + val plan2 = join2.queryExecution.executedPlan + assert(collect(plan2) { case s: SortMergeJoinExec => s }.size == 1) + assert(collect(plan2) { case b: BroadcastHashJoinExec => b }.size == 1) + assert(collect(plan2) { case e: ShuffleExchangeExec => e }.isEmpty) + + // Validate the data with broadcast join off. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = join1.join(t3, join1("i2") === t3("i3") && join1("j2") === t3("j3")) + checkAnswer(join2, df) + } + } + } + } + + test("broadcast join where streamed side's output partitioning is PartitioningCollection") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val t1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1") + val t2 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i2", "j2") + val t3 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i3", "j3") + val t4 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i4", "j4") + + // join1 is a sort merge join (shuffle on the both sides). + val join1 = t1.join(t2, t1("i1") === t2("i2")) + val plan1 = join1.queryExecution.executedPlan + assert(collect(plan1) { case s: SortMergeJoinExec => s }.size == 1) + assert(collect(plan1) { case e: ShuffleExchangeExec => e }.size == 2) + + // join2 is a broadcast join where t3 is broadcasted. Note that output partitioning on the + // streamed side (join1) is PartitioningCollection (sort merge join) + val join2 = join1.join(t3, join1("i1") === t3("i3")) + val plan2 = join2.queryExecution.executedPlan + assert(collect(plan2) { case s: SortMergeJoinExec => s }.size == 1) + assert(collect(plan2) { case e: ShuffleExchangeExec => e }.size == 2) + val broadcastJoins = collect(plan2) { case b: BroadcastHashJoinExec => b } + assert(broadcastJoins.size == 1) + assert(broadcastJoins(0).outputPartitioning.isInstanceOf[PartitioningCollection]) + val p = broadcastJoins(0).outputPartitioning.asInstanceOf[PartitioningCollection] + assert(p.partitionings.size == 3) + // Verify all the combinations of output partitioning. + Seq(Seq(t1("i1")), Seq(t2("i2")), Seq(t3("i3"))).foreach { expected => + val expectedExpressions = expected.map(_.expr) + assert(p.partitionings.exists { + case h: HashPartitioning => expressionsEqual(h.expressions, expectedExpressions) + }) + } + + // Join on the column from the broadcasted side (i3) and make sure output partitioning + // is maintained by checking no shuffle exchange is introduced. Note that one extra + // ShuffleExchangeExec is from t4, not from join2. + val join3 = join2.join(t4, join2("i3") === t4("i4")) + val plan3 = join3.queryExecution.executedPlan + assert(collect(plan3) { case s: SortMergeJoinExec => s }.size == 2) + assert(collect(plan3) { case b: BroadcastHashJoinExec => b }.size == 1) + assert(collect(plan3) { case e: ShuffleExchangeExec => e }.size == 3) + + // Validate the data with broadcast join off. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = join2.join(t4, join2("i3") === t4("i4")) + checkAnswer(join3, df) + } + } + } + + test("BroadcastHashJoinExec output partitioning scenarios for inner join") { + val l1 = AttributeReference("l1", LongType)() + val l2 = AttributeReference("l2", LongType)() + val l3 = AttributeReference("l3", LongType)() + val r1 = AttributeReference("r1", LongType)() + val r2 = AttributeReference("r2", LongType)() + val r3 = AttributeReference("r3", LongType)() + + // Streamed side has a HashPartitioning. + var bhj = BroadcastHashJoinExec( + leftKeys = Seq(l2, l3), + rightKeys = Seq(r1, r2), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(l1, l2, l3), 1)), + right = DummySparkPlan()) + var expected = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2, l3), 1), + HashPartitioning(Seq(l1, l2, r2), 1), + HashPartitioning(Seq(l1, r1, l3), 1), + HashPartitioning(Seq(l1, r1, r2), 1))) + assert(bhj.outputPartitioning === expected) + + // Streamed side has a PartitioningCollection. + bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l2, l3), + rightKeys = Seq(r1, r2, r3), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2), 1), HashPartitioning(Seq(l3), 1)))), + right = DummySparkPlan()) + expected = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2), 1), + HashPartitioning(Seq(l1, r2), 1), + HashPartitioning(Seq(r1, l2), 1), + HashPartitioning(Seq(r1, r2), 1), + HashPartitioning(Seq(l3), 1), + HashPartitioning(Seq(r3), 1))) + assert(bhj.outputPartitioning === expected) + + // Streamed side has a nested PartitioningCollection. + bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l2, l3), + rightKeys = Seq(r1, r2, r3), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + PartitioningCollection(Seq(HashPartitioning(Seq(l1), 1), HashPartitioning(Seq(l2), 1))), + HashPartitioning(Seq(l3), 1)))), + right = DummySparkPlan()) + expected = PartitioningCollection(Seq( + PartitioningCollection(Seq( + HashPartitioning(Seq(l1), 1), + HashPartitioning(Seq(r1), 1), + HashPartitioning(Seq(l2), 1), + HashPartitioning(Seq(r2), 1))), + HashPartitioning(Seq(l3), 1), + HashPartitioning(Seq(r3), 1))) + assert(bhj.outputPartitioning === expected) + + // One-to-mapping case ("l1" = "r1" AND "l1" = "r2") + bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l1), + rightKeys = Seq(r1, r2), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(l1, l2), 1)), + right = DummySparkPlan()) + expected = PartitioningCollection(Seq( + HashPartitioning(Seq(l1, l2), 1), + HashPartitioning(Seq(r1, l2), 1), + HashPartitioning(Seq(r2, l2), 1))) + assert(bhj.outputPartitioning === expected) + } + + test("BroadcastHashJoinExec output partitioning size should be limited with a config") { + val l1 = AttributeReference("l1", LongType)() + val l2 = AttributeReference("l2", LongType)() + val r1 = AttributeReference("r1", LongType)() + val r2 = AttributeReference("r2", LongType)() + + val expected = Seq( + HashPartitioning(Seq(l1, l2), 1), + HashPartitioning(Seq(l1, r2), 1), + HashPartitioning(Seq(r1, l2), 1), + HashPartitioning(Seq(r1, r2), 1)) + + Seq(1, 2, 3, 4).foreach { limit => + withSQLConf( + SQLConf.BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT.key -> s"$limit") { + val bhj = BroadcastHashJoinExec( + leftKeys = Seq(l1, l2), + rightKeys = Seq(r1, r2), + Inner, + BuildRight, + None, + left = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(l1, l2), 1)), + right = DummySparkPlan()) + assert(bhj.outputPartitioning === PartitioningCollection(expected.take(limit))) + } + } + } + + private def expressionsEqual(l: Seq[Expression], r: Seq[Expression]): Boolean = { + l.length == r.length && l.zip(r).forall { case (e1, e2) => e1.semanticEquals(e2) } + } } class BroadcastJoinSuite extends BroadcastJoinSuiteBase with DisableAdaptiveExecutionSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index 554990413c28c..e8ac09fdb634e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -200,6 +200,14 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession { singleConditionEQ, Seq(Row(2, 1.0), Row(2, 1.0), Row(3, 3.0), Row(6, null))) + testExistenceJoin( + "test single unique condition (equal) for left semi join", + LeftSemi, + left, + right.select(right.col("c")).distinct(), /* Trigger BHJs and SHJs unique key code path! */ + singleConditionEQ, + Seq(Row(2, 1.0), Row(2, 1.0), Row(3, 3.0), Row(6, null))) + testExistenceJoin( "test composed condition (equal & non-equal) for left semi join", LeftSemi, @@ -229,7 +237,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession { "test single unique condition (equal) for left Anti join", LeftAnti, left, - right.select(right.col("c")).distinct(), /* Trigger BHJs unique key code path! */ + right.select(right.col("c")).distinct(), /* Trigger BHJs and SHJs unique key code path! */ singleConditionEQ, Seq(Row(1, 2.0), Row(1, 2.0), Row(null, null), Row(null, 5.0))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 3526aa254c280..21ee88f0d7426 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -144,6 +144,7 @@ class HashedRelationSuite extends SharedSparkSession { } val longRelation2 = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) + .asInstanceOf[LongHashedRelation] assert(!longRelation2.keyIsUnique) (0 until 100).foreach { i => val rows = longRelation2.get(i).toArray diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 879f282e4d05d..a466e05816ad8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -47,7 +47,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { sparkContext.parallelize(Seq( Row(0, 0.0), Row(2, 3.0), // This row is duplicated to ensure that we will have multiple buffered matches - Row(2, -1.0), + Row(2, -1.0), // This row is duplicated to ensure that we will have multiple buffered matches Row(2, -1.0), Row(2, 3.0), Row(3, 2.0), @@ -62,6 +62,32 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { LessThan(left.col("b").expr, right.col("d").expr)) } + private lazy val uniqueLeft = spark.createDataFrame( + sparkContext.parallelize(Seq( + Row(1, 2.0), + Row(2, 1.0), + Row(3, 3.0), + Row(5, 1.0), + Row(6, 6.0), + Row(null, null) + )), new StructType().add("a", IntegerType).add("b", DoubleType)) + + private lazy val uniqueRight = spark.createDataFrame( + sparkContext.parallelize(Seq( + Row(0, 0.0), + Row(2, 3.0), + Row(3, 2.0), + Row(4, 1.0), + Row(5, 3.0), + Row(7, 7.0), + Row(null, null) + )), new StructType().add("c", IntegerType).add("d", DoubleType)) + + private lazy val uniqueCondition = { + And((uniqueLeft.col("a") === uniqueRight.col("c")).expr, + LessThan(uniqueLeft.col("b").expr, uniqueRight.col("d").expr)) + } + // Note: the input dataframes and expression must be evaluated lazily because // the SQLContext should be used only within a test to keep SQL tests stable private def testOuterJoin( @@ -243,4 +269,39 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { condition, Seq.empty ) + + // --- Join keys are unique --------------------------------------------------------------------- + + testOuterJoin( + "left outer join with unique keys", + uniqueLeft, + uniqueRight, + LeftOuter, + uniqueCondition, + Seq( + (null, null, null, null), + (1, 2.0, null, null), + (2, 1.0, 2, 3.0), + (3, 3.0, null, null), + (5, 1.0, 5, 3.0), + (6, 6.0, null, null) + ) + ) + + testOuterJoin( + "right outer join with unique keys", + uniqueLeft, + uniqueRight, + RightOuter, + uniqueCondition, + Seq( + (null, null, null, null), + (null, null, 0, 0.0), + (2, 1.0, 2, 3.0), + (null, null, 3, 2.0), + (null, null, 4, 1.0), + (5, 1.0, 5, 3.0), + (null, null, 7, 7.0) + ) + ) } 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 50652690339a8..078a3ba029e4b 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 @@ -346,8 +346,8 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils val rightDf = (1 to 10).map(i => (i, i.toString)).toSeq.toDF("key2", "value") Seq((0L, "right_outer", leftDf, rightDf, 10L, false), (0L, "left_outer", rightDf, leftDf, 10L, false), - (0L, "right_outer", leftDf, rightDf, 10L, true), - (0L, "left_outer", rightDf, leftDf, 10L, true), + (1L, "right_outer", leftDf, rightDf, 10L, true), + (1L, "left_outer", rightDf, leftDf, 10L, true), (2L, "left_anti", rightDf, leftDf, 8L, true), (2L, "left_semi", rightDf, leftDf, 2L, true), (1L, "left_anti", rightDf, leftDf, 8L, false), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index f1cd37f91d78a..5701cc917258b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.sql.internal +import java.util.TimeZone + import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.log4j.Level import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.MIT import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext} @@ -112,7 +115,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") assert(spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) === 10) } finally { - sql(s"set ${SQLConf.SHUFFLE_PARTITIONS}=$original") + sql(s"set ${SQLConf.SHUFFLE_PARTITIONS.key}=$original") } } @@ -139,11 +142,14 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES).isEmpty) sql(s"reset") assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL)) assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) === + Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation")) } finally { - sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=$original") } } @@ -159,7 +165,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) } finally { - sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=$original") } } @@ -178,6 +184,45 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } + test("SPARK-32406: reset - single configuration") { + spark.sessionState.conf.clear() + // spark core conf w/o entry registered + val appId = spark.sparkContext.getConf.getAppId + sql("RESET spark.app.id") + assert(spark.conf.get("spark.app.id") === appId, "Should not change spark core ones") + // spark core conf w/ entry registered + val e1 = intercept[AnalysisException](sql("RESET spark.executor.cores")) + assert(e1.getMessage === "Cannot modify the value of a Spark config: spark.executor.cores;") + + // user defined settings + sql("SET spark.abc=xyz") + assert(spark.conf.get("spark.abc") === "xyz") + sql("RESET spark.abc") + intercept[NoSuchElementException](spark.conf.get("spark.abc")) + sql("RESET spark.abc") // ignore nonexistent keys + + // runtime sql configs + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + sql(s"SET ${SQLConf.GROUP_BY_ORDINAL.key}=false") + sql(s"RESET ${SQLConf.GROUP_BY_ORDINAL.key}") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === original) + + // runtime sql configs with optional defaults + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES).isEmpty) + sql(s"RESET ${SQLConf.OPTIMIZER_EXCLUDED_RULES.key}") + assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) === + Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation")) + sql(s"SET ${SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES.key}=abc") + sql(s"RESET ${SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES.key}") + assert(spark.conf.get(SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES).isEmpty) + + // static sql configs + val e2 = intercept[AnalysisException](sql(s"RESET ${StaticSQLConf.WAREHOUSE_PATH.key}")) + assert(e2.getMessage === + s"Cannot modify the value of a static config: ${StaticSQLConf.WAREHOUSE_PATH.key};") + + } + test("invalid conf value") { spark.sessionState.conf.clear() val e = intercept[IllegalArgumentException] { @@ -383,4 +428,29 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } assert(e.getMessage === "Cannot resolve the given timezone with ZoneId.of(_, ZoneId.SHORT_IDS)") } + + test("set time zone") { + TimeZone.getAvailableIDs().foreach { zid => + sql(s"set time zone '$zid'") + assert(spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) === zid) + } + sql("set time zone local") + assert(spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) === TimeZone.getDefault.getID) + + val e1 = intercept[IllegalArgumentException](sql("set time zone 'invalid'")) + assert(e1.getMessage === "Cannot resolve the given timezone with" + + " ZoneId.of(_, ZoneId.SHORT_IDS)") + + (-18 to 18).map(v => (v, s"interval '$v' hours")).foreach { case (i, interval) => + sql(s"set time zone $interval") + val zone = spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) + if (i == 0) { + assert(zone === "Z") + } else { + assert(zone === String.format("%+03d:00", new Integer(i))) + } + } + val e2 = intercept[ParseException](sql("set time zone interval 19 hours")) + assert(e2.getMessage contains "The interval value must be in the range of [-18, +18] hours") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala new file mode 100644 index 0000000000000..46bdb1918147c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCNestedDataSourceSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import org.apache.spark.sql.NestedDataSourceSuiteBase +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils + +class JDBCNestedDataSourceSuite extends NestedDataSourceSuiteBase { + override val nestedDataSources: Seq[String] = Seq("jdbc") + private val tempDir = Utils.createTempDir() + private val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" + override val colType: String = "in the customSchema option value" + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDir) + super.afterAll() + } + + override def readOptions(schema: StructType): Map[String, String] = { + Map("url" -> url, "dbtable" -> "t1", "customSchema" -> schema.toDDL) + } + + override def save(selectExpr: Seq[String], format: String, path: String): Unit = { + // We ignore `selectExpr` because: + // 1. H2 doesn't support nested columns + // 2. JDBC datasource checks duplicates before comparing of user's schema with + // actual schema of `t1`. + spark + .range(1L) + .write.mode("overwrite") + .options(Map("url" -> url, "dbtable" -> "t1")) + .format(format) + .save() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 7c723f083642d..06dd6615a817b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -21,6 +21,8 @@ import java.math.BigDecimal import java.sql.{Date, DriverManager, SQLException, Timestamp} import java.util.{Calendar, GregorianCalendar, Properties} +import scala.collection.JavaConverters._ + import org.h2.jdbc.JdbcSQLException import org.scalatest.{BeforeAndAfter, PrivateMethodTester} @@ -71,7 +73,8 @@ class JDBCSuite extends QueryTest } } - before { + override def beforeAll(): Unit = { + super.beforeAll() Utils.classForName("org.h2.Driver") // Extra properties that will be specified for our database. We need these to test // usage of parameters from OPTIONS clause in queries. @@ -273,8 +276,9 @@ class JDBCSuite extends QueryTest // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. } - after { + override def afterAll(): Unit = { conn.close() + super.afterAll() } // Check whether the tables are fetched in the expected degree of parallelism @@ -1180,7 +1184,10 @@ class JDBCSuite extends QueryTest test("SPARK-16387: Reserved SQL words are not escaped by JDBC writer") { val df = spark.createDataset(Seq("a", "b", "c")).toDF("order") - val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") + val schema = JdbcUtils.schemaString( + df.schema, + df.sqlContext.conf.caseSensitiveAnalysis, + "jdbc:mysql://localhost:3306/temp") assert(schema.contains("`order` TEXT")) } @@ -1302,7 +1309,8 @@ class JDBCSuite extends QueryTest testJdbcOptions(new JDBCOptions(parameters)) testJdbcOptions(new JDBCOptions(CaseInsensitiveMap(parameters))) // test add/remove key-value from the case-insensitive map - var modifiedParameters = CaseInsensitiveMap(Map.empty) ++ parameters + var modifiedParameters = + (CaseInsensitiveMap(Map.empty) ++ parameters).asInstanceOf[Map[String, String]] testJdbcOptions(new JDBCOptions(modifiedParameters)) modifiedParameters -= "dbtable" assert(modifiedParameters.get("dbTAblE").isEmpty) @@ -1715,4 +1723,23 @@ class JDBCSuite extends QueryTest jdbcDF = sqlContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts) checkAnswer(jdbcDF, Row("mary", 2) :: Nil) } + + test("SPARK-32364: JDBCOption constructor") { + val extraOptions = CaseInsensitiveMap[String](Map("UrL" -> "url1", "dBTable" -> "table1")) + val connectionProperties = new Properties() + connectionProperties.put("url", "url2") + connectionProperties.put("dbtable", "table2") + + // connection property should override the options in extraOptions + val params = extraOptions ++ connectionProperties.asScala + assert(params.size == 2) + assert(params.get("uRl").contains("url2")) + assert(params.get("DbtaBle").contains("table2")) + + // JDBCOptions constructor parameter should overwrite the existing conf + val options = new JDBCOptions(url, "table3", params) + assert(options.asProperties.size == 2) + assert(options.asProperties.get("url") == url) + assert(options.asProperties.get("dbtable") == "table3") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 8021ef1a17a18..3f621e04338a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -390,7 +390,11 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { val expectedSchemaStr = colTypes.map { case (col, dataType) => s""""$col" $dataType """ }.mkString(", ") - assert(JdbcUtils.schemaString(df, url1, Option(createTableColTypes)) == expectedSchemaStr) + assert(JdbcUtils.schemaString( + df.schema, + df.sqlContext.conf.caseSensitiveAnalysis, + url1, + Option(createTableColTypes)) == expectedSchemaStr) } testCreateTableColDataTypes(Seq("boolean")) 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 b6767eb3132ea..98886d271e977 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 @@ -876,7 +876,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } test("bucket coalescing eliminates shuffle") { - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. // Currently, sort will be introduced for the side that is coalesced. val testSpec1 = BucketedTableTestSpec( @@ -911,7 +911,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "false") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "false") { // Coalescing buckets is disabled by a config. run( BucketedTableTestSpec( @@ -921,8 +921,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } withSQLConf( - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_MAX_BUCKET_RATIO.key -> "2") { + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO.key -> "2") { // Coalescing buckets is not applied because the ratio of the number of buckets (3) // is greater than max allowed (2). run( @@ -932,7 +932,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { Some(BucketSpec(4, Seq("i", "j"), Seq("i", "j"))), expectedShuffle = true)) } - withSQLConf(SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + withSQLConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { run( // Coalescing buckets is not applied because the bigger number of buckets (8) is not // divisible by the smaller number of buckets (7). @@ -950,7 +950,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { withSQLConf( SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", - SQLConf.COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key -> "true") { + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true") { def verify( query: String, expectedNumShuffles: Int, @@ -964,7 +964,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } if (expectedCoalescedNumBuckets.isDefined) { assert(scans.length == 1) - assert(scans(0).optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + assert(scans.head.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) } else { assert(scans.isEmpty) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 705e980df86a0..67ab72a79145e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -24,7 +24,9 @@ import java.util.{Calendar, Date, Locale} import java.util.concurrent.TimeUnit._ import org.apache.commons.io.FileUtils -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, Dataset} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala index 63b5792ebd515..82aa1453f9ba2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.streaming.ui import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala index 46f2eadc05835..fbf4dcead4a9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.streaming.ui import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.streaming.StreamingQueryProgress diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index fe0a8439acc2d..c7ca0125c8a0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -224,6 +224,28 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with assert(LastOptions.parameters("opt3") == "3") } + test("SPARK-32364: path argument of load function should override all existing options") { + spark.read + .format("org.apache.spark.sql.test") + .option("paTh", "1") + .option("PATH", "2") + .option("Path", "3") + .option("patH", "4") + .load("5") + assert(LastOptions.parameters("path") == "5") + } + + test("SPARK-32364: path argument of save function should override all existing options") { + Seq(1).toDF.write + .format("org.apache.spark.sql.test") + .option("paTh", "1") + .option("PATH", "2") + .option("Path", "3") + .option("patH", "4") + .save("5") + assert(LastOptions.parameters("path") == "5") + } + test("pass partitionBy as options") { Seq(1).toDF.write .format("org.apache.spark.sql.test") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala index 2861b80190abe..dfa0348f1227c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.sql.test -import org.scalatest.FlatSpec - -import org.apache.spark.sql.Dataset +import org.scalatest.flatspec.AnyFlatSpec /** * The purpose of this suite is to make sure that generic FlatSpec-based scala * tests work with a shared spark session */ -class GenericFlatSpecSuite extends FlatSpec with SharedSparkSessionBase { +class GenericFlatSpecSuite extends AnyFlatSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala index efdaac3ae1f0c..1b6724054a3ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.test -import org.scalatest.FunSpec +import org.scalatest.funspec.AnyFunSpec import org.apache.spark.sql.Dataset @@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset * The purpose of this suite is to make sure that generic FunSpec-based scala * tests work with a shared spark session */ -class GenericFunSpecSuite extends FunSpec with SharedSparkSessionBase { +class GenericFunSpecSuite extends AnyFunSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala index 3a3540c1dbdbf..e693dc9c9b134 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.sql.test -import org.scalatest.WordSpec - -import org.apache.spark.sql.Dataset +import org.scalatest.wordspec.AnyWordSpec /** * The purpose of this suite is to make sure that generic WordSpec-based scala * tests work with a shared spark session */ -class GenericWordSpecSuite extends WordSpec with SharedSparkSessionBase { +class GenericWordSpecSuite extends AnyWordSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5bf20b209aff7..0d9d7515d85ed 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -152,6 +152,18 @@ + + add-test-source + generate-sources + + add-test-source + + + + v${hive.version.short}/src/test/scala + + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 57ed15a76a893..922af72604027 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -19,11 +19,9 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction import java.util.{Arrays, Map => JMap} -import java.util.concurrent.RejectedExecutionException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.util.control.NonFatal import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.shims.Utils @@ -35,9 +33,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.VariableSubstitution import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval -import org.apache.spark.util.{Utils => SparkUtils} private[hive] class SparkExecuteStatementOperation( val sqlContext: SQLContext, @@ -112,7 +110,7 @@ private[hive] class SparkExecuteStatementOperation( } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = withLocalProperties { - log.info(s"Received getNextRowSet request order=${order} and maxRowsL=${maxRowsL} " + + logInfo(s"Received getNextRowSet request order=${order} and maxRowsL=${maxRowsL} " + s"with ${statementId}") validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) @@ -181,7 +179,7 @@ private[hive] class SparkExecuteStatementOperation( resultOffset += 1 } previousFetchEndOffset = resultOffset - log.info(s"Returning result set with ${curRow} rows from offsets " + + logInfo(s"Returning result set with ${curRow} rows from offsets " + s"[$previousFetchStartOffset, $previousFetchEndOffset) with $statementId") resultRowSet } @@ -220,7 +218,7 @@ private[hive] class SparkExecuteStatementOperation( } catch { case e: HiveSQLException => setOperationException(e) - log.error("Error running hive query: ", e) + logError(s"Error executing query with $statementId,", e) } } } @@ -240,21 +238,7 @@ private[hive] class SparkExecuteStatementOperation( val backgroundHandle = parentSession.getSessionManager().submitBackgroundOperation(backgroundOperation) setBackgroundHandle(backgroundHandle) - } catch { - case rejected: RejectedExecutionException => - logError("Error submitting query in background, query rejected", rejected) - setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, rejected.getMessage, SparkUtils.exceptionString(rejected)) - throw new HiveSQLException("The background threadpool cannot accept" + - " new task for execution, please retry the operation", rejected) - case NonFatal(e) => - logError(s"Error executing query in background", e) - setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw new HiveSQLException(e) - } + } catch onError() } } @@ -278,7 +262,8 @@ private[hive] class SparkExecuteStatementOperation( parentSession.getSessionState.getConf.setClassLoader(executionHiveClassLoader) } - sqlContext.sparkContext.setJobGroup(statementId, statement) + val substitutorStatement = new VariableSubstitution(sqlContext.conf).substitute(statement) + sqlContext.sparkContext.setJobGroup(statementId, substitutorStatement) result = sqlContext.sql(statement) logDebug(result.queryExecution.toString()) HiveThriftServer2.eventManager.onStatementParsed(statementId, @@ -294,30 +279,7 @@ private[hive] class SparkExecuteStatementOperation( } dataTypes = result.schema.fields.map(_.dataType) } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - // When cancel() or close() is called very quickly after the query is started, - // then they may both call cleanup() before Spark Jobs are started. But before background - // task interrupted, it may have start some spark job, so we need to cancel again to - // make sure job was cancelled when background thread was interrupted - if (statementId != null) { - sqlContext.sparkContext.cancelJobGroup(statementId) - } - val currentState = getStatus().getState() - if (currentState.isTerminal) { - // This may happen if the execution was cancelled, and then closed from another thread. - logWarning(s"Ignore exception in terminal state with $statementId: $e") - } else { - logError(s"Error executing query with $statementId, currentState $currentState, ", e) - setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - e match { - case _: HiveSQLException => throw e - case _ => throw new HiveSQLException("Error running query: " + e.toString, e) - } - } + onError(needCancel = true) } finally { synchronized { if (!getStatus.getState.isTerminal) { @@ -348,9 +310,7 @@ private[hive] class SparkExecuteStatementOperation( } } // RDDs will be cleaned automatically upon garbage collection. - if (statementId != null) { - sqlContext.sparkContext.cancelJobGroup(statementId) - } + sqlContext.sparkContext.cancelJobGroup(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index d42732f426681..069517acd68cc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.UUID import java.util.regex.Pattern import scala.collection.JavaConverters.seqAsJavaListConverter -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObject} import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.hive.service.cli._ @@ -34,7 +32,7 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils.toJavaSQLType -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ /** * Spark's own SparkGetColumnsOperation @@ -126,12 +124,52 @@ private[hive] class SparkGetColumnsOperation( HiveThriftServer2.eventManager.onStatementFinish(statementId) } + /** + * For boolean, numeric and datetime types, it returns the default size of its catalyst type + * For struct type, when its elements are fixed-size, the summation of all element sizes will be + * returned. + * For array, map, string, and binaries, the column size is variable, return null as unknown. + */ + private def getColumnSize(typ: DataType): Option[Int] = typ match { + case dt @ (BooleanType | _: NumericType | DateType | TimestampType) => Some(dt.defaultSize) + case StructType(fields) => + val sizeArr = fields.map(f => getColumnSize(f.dataType)) + if (sizeArr.contains(None)) { + None + } else { + Some(sizeArr.map(_.get).sum) + } + case other => None + } + + /** + * The number of fractional digits for this type. + * Null is returned for data types where this is not applicable. + * For boolean and integrals, the decimal digits is 0 + * For floating types, we follow the IEEE Standard for Floating-Point Arithmetic (IEEE 754) + * For timestamp values, we support microseconds + * For decimals, it returns the scale + */ + private def getDecimalDigits(typ: DataType) = typ match { + case BooleanType | _: IntegerType => Some(0) + case FloatType => Some(7) + case DoubleType => Some(15) + case d: DecimalType => Some(d.scale) + case TimestampType => Some(6) + case _ => None + } + + private def getNumPrecRadix(typ: DataType): Option[Int] = typ match { + case _: NumericType => Some(10) + case _ => None + } + private def addToRowSet( columnPattern: Pattern, dbName: String, tableName: String, schema: StructType): Unit = { - schema.foreach { column => + schema.zipWithIndex.foreach { case (column, pos) => if (columnPattern != null && !columnPattern.matcher(column.name).matches()) { } else { val rowData = Array[AnyRef]( @@ -141,17 +179,17 @@ private[hive] class SparkGetColumnsOperation( column.name, // COLUMN_NAME toJavaSQLType(column.dataType.sql).asInstanceOf[AnyRef], // DATA_TYPE column.dataType.sql, // TYPE_NAME - null, // COLUMN_SIZE + getColumnSize(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // COLUMN_SIZE null, // BUFFER_LENGTH, unused - null, // DECIMAL_DIGITS - null, // NUM_PREC_RADIX + getDecimalDigits(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // DECIMAL_DIGITS + getNumPrecRadix(column.dataType).map(_.asInstanceOf[AnyRef]).orNull, // NUM_PREC_RADIX (if (column.nullable) 1 else 0).asInstanceOf[AnyRef], // NULLABLE column.getComment().getOrElse(""), // REMARKS null, // COLUMN_DEF null, // SQL_DATA_TYPE null, // SQL_DATETIME_SUB null, // CHAR_OCTET_LENGTH - null, // ORDINAL_POSITION + pos.asInstanceOf[AnyRef], // ORDINAL_POSITION "YES", // IS_NULLABLE null, // SCOPE_CATALOG null, // SCOPE_SCHEMA diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 446669d08e76b..8e8b2d7ff774d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.thriftserver +import java.util.concurrent.RejectedExecutionException + import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.Operation @@ -37,7 +39,7 @@ private[hive] trait SparkOperation extends Operation with Logging { protected var statementId = getHandle().getHandleIdentifier().getPublicId().toString() - protected def cleanup(): Unit = Unit // noop by default + protected def cleanup(): Unit = () // noop by default abstract override def run(): Unit = { withLocalProperties { @@ -94,15 +96,32 @@ private[hive] trait SparkOperation extends Operation with Logging { throw new IllegalArgumentException(s"Unknown table type is found: $t") } - protected def onError(): PartialFunction[Throwable, Unit] = { + protected def onError(needCancel: Boolean = false): PartialFunction[Throwable, Unit] = { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. case e: Throwable => - logError(s"Error executing get catalogs operation with $statementId", e) - super.setState(OperationState.ERROR) - HiveThriftServer2.eventManager.onStatementError( - statementId, e.getMessage, Utils.exceptionString(e)) - e match { - case _: HiveSQLException => throw e - case _ => throw new HiveSQLException(s"Error operating $getType ${e.getMessage}", e) + // When cancel() or close() is called very quickly after the query is started, + // then they may both call cleanup() before Spark Jobs are started. But before background + // task interrupted, it may have start some spark job, so we need to cancel again to + // make sure job was cancelled when background thread was interrupted + if (needCancel) sqlContext.sparkContext.cancelJobGroup(statementId) + val currentState = getStatus.getState + if (currentState.isTerminal) { + // This may happen if the execution was cancelled, and then closed from another thread. + logWarning(s"Ignore exception in terminal state with $statementId: $e") + } else { + super.setState(OperationState.ERROR) + HiveThriftServer2.eventManager.onStatementError( + statementId, e.getMessage, Utils.exceptionString(e)) + e match { + case _: HiveSQLException => throw e + case rejected: RejectedExecutionException => + throw new HiveSQLException("The background threadpool cannot accept" + + " new task for execution, please retry the operation", rejected) + case _ => + val tips = if (shouldRunAsync()) " in background" else "" + throw new HiveSQLException(s"Error operating $getType$tips: ${e.getMessage}", e) + } } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 12fba0eae6dce..8faeee523d983 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.internal.VariableSubstitution private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlContext) @@ -59,7 +60,8 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont override def run(command: String): CommandProcessorResponse = { // TODO unify the error code try { - context.sparkContext.setJobDescription(command) + val substitutorCommand = new VariableSubstitution(context.conf).substitute(command) + context.sparkContext.setJobDescription(substitutorCommand) val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) hiveResponse = SQLExecution.withNewExecutionId(execution) { hiveResultString(execution.executedPlan) diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties index 21975ba818142..a73946c6dc5f3 100644 --- a/sql/hive-thriftserver/src/test/resources/log4j.properties +++ b/sql/hive-thriftserver/src/test/resources/log4j.properties @@ -63,3 +63,5 @@ log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR # Parquet related logging log4j.logger.org.apache.parquet.CorruptStatistics=ERROR log4j.logger.parquet.CorruptStatistics=ERROR + +log4j.logger.org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation=OFF diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala index 42d86e98a7273..47db7e34a5a2c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.hive.thriftserver import java.lang.reflect.InvocationTargetException -import java.nio.ByteBuffer -import java.util.UUID import scala.collection.JavaConverters._ import scala.collection.mutable @@ -27,7 +25,6 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hive.service.cli.OperationHandle import org.apache.hive.service.cli.operation.{GetCatalogsOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl, SessionManager} -import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType} import org.apache.spark.SparkFunSuite @@ -65,31 +62,6 @@ class HiveSessionImplSuite extends SparkFunSuite { } } -class GetCatalogsOperationMock(parentSession: HiveSession) - extends GetCatalogsOperation(parentSession) { - - override def runInternal(): Unit = {} - - override def getHandle: OperationHandle = { - val uuid: UUID = UUID.randomUUID() - val tHandleIdentifier: THandleIdentifier = new THandleIdentifier() - tHandleIdentifier.setGuid(getByteBufferFromUUID(uuid)) - tHandleIdentifier.setSecret(getByteBufferFromUUID(uuid)) - val tOperationHandle: TOperationHandle = new TOperationHandle() - tOperationHandle.setOperationId(tHandleIdentifier) - tOperationHandle.setOperationType(TOperationType.GET_TYPE_INFO) - tOperationHandle.setHasResultSetIsSet(false) - new OperationHandle(tOperationHandle) - } - - private def getByteBufferFromUUID(uuid: UUID): Array[Byte] = { - val bb: ByteBuffer = ByteBuffer.wrap(new Array[Byte](16)) - bb.putLong(uuid.getMostSignificantBits) - bb.putLong(uuid.getLeastSignificantBits) - bb.array - } -} - class OperationManagerMock extends OperationManager { private val calledHandles: mutable.Set[OperationHandle] = new mutable.HashSet[OperationHandle]() @@ -114,3 +86,4 @@ class OperationManagerMock extends OperationManager { def getCalledHandles: mutable.Set[OperationHandle] = calledHandles } + diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index f7ee3e0a46cd1..5df337044480e 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{DatabaseMetaData, ResultSet} +import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, DecimalType, DoubleType, FloatType, IntegerType, MapType, NumericType, StringType, StructType, TimestampType} + class SparkMetadataOperationSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.binary @@ -247,4 +249,88 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { checkResult(metaData.getTypeInfo, ThriftserverShimUtils.supportedType().map(_.getName)) } } + + test("check results from get columns operation from thrift server") { + val schemaName = "default" + val tableName = "spark_get_col_operation" + val schema = new StructType() + .add("c0", "boolean", nullable = false, "0") + .add("c1", "tinyint", nullable = true, "1") + .add("c2", "smallint", nullable = false, "2") + .add("c3", "int", nullable = true, "3") + .add("c4", "long", nullable = false, "4") + .add("c5", "float", nullable = true, "5") + .add("c6", "double", nullable = false, "6") + .add("c7", "decimal(38, 20)", nullable = true, "7") + .add("c8", "decimal(10, 2)", nullable = false, "8") + .add("c9", "string", nullable = true, "9") + .add("c10", "array", nullable = false, "10") + .add("c11", "array", nullable = true, "11") + .add("c12", "map", nullable = false, "12") + .add("c13", "date", nullable = true, "13") + .add("c14", "timestamp", nullable = false, "14") + .add("c15", "struct", nullable = true, "15") + .add("c16", "binary", nullable = false, "16") + + val ddl = + s""" + |CREATE TABLE $schemaName.$tableName ( + | ${schema.toDDL} + |) + |using parquet""".stripMargin + + withJdbcStatement(tableName) { statement => + statement.execute(ddl) + + val databaseMetaData = statement.getConnection.getMetaData + val rowSet = databaseMetaData.getColumns("", schemaName, tableName, null) + + import java.sql.Types._ + val expectedJavaTypes = Seq(BOOLEAN, TINYINT, SMALLINT, INTEGER, BIGINT, FLOAT, DOUBLE, + DECIMAL, DECIMAL, VARCHAR, ARRAY, ARRAY, JAVA_OBJECT, DATE, TIMESTAMP, STRUCT, BINARY) + + var pos = 0 + + while (rowSet.next()) { + assert(rowSet.getString("TABLE_CAT") === null) + assert(rowSet.getString("TABLE_SCHEM") === schemaName) + assert(rowSet.getString("TABLE_NAME") === tableName) + assert(rowSet.getString("COLUMN_NAME") === schema(pos).name) + assert(rowSet.getInt("DATA_TYPE") === expectedJavaTypes(pos)) + assert(rowSet.getString("TYPE_NAME") === schema(pos).dataType.sql) + + val colSize = rowSet.getInt("COLUMN_SIZE") + schema(pos).dataType match { + case StringType | BinaryType | _: ArrayType | _: MapType => assert(colSize === 0) + case o => assert(colSize === o.defaultSize) + } + + assert(rowSet.getInt("BUFFER_LENGTH") === 0) // not used + val decimalDigits = rowSet.getInt("DECIMAL_DIGITS") + schema(pos).dataType match { + case BooleanType | _: IntegerType => assert(decimalDigits === 0) + case d: DecimalType => assert(decimalDigits === d.scale) + case FloatType => assert(decimalDigits === 7) + case DoubleType => assert(decimalDigits === 15) + case TimestampType => assert(decimalDigits === 6) + case _ => assert(decimalDigits === 0) // nulls + } + + val radix = rowSet.getInt("NUM_PREC_RADIX") + schema(pos).dataType match { + case _: NumericType => assert(radix === 10) + case _ => assert(radix === 0) // nulls + } + + assert(rowSet.getInt("NULLABLE") === 1) + assert(rowSet.getString("REMARKS") === pos.toString) + assert(rowSet.getInt("ORDINAL_POSITION") === pos) + assert(rowSet.getString("IS_NULLABLE") === "YES") + assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") + pos += 1 + } + + assert(pos === 17, "all columns should have been verified") + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index fd3a638c4fa44..102fd77c06f3a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -21,6 +21,9 @@ import java.sql.SQLException import org.apache.hive.service.cli.HiveSQLException +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.types._ + trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("the scratch dir will be deleted during server start but recreated with new operation") { @@ -52,31 +55,51 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("Full stack traces as error message for jdbc or thrift client") { val sql = "select date_sub(date'2011-11-11', '1.2')" - withCLIServiceClient { client => - val sessionHandle = client.openSession(user, "") + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] - val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] - val e = intercept[HiveSQLException] { - client.executeStatement( - sessionHandle, - sql, - confOverlay) + withSQLConf((HiveUtils.HIVE_THRIFT_SERVER_ASYNC.key, "false")) { + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + val e = intercept[HiveSQLException] { + client.executeStatement(sessionHandle, sql, confOverlay) + } + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(!e.getMessage + .contains("java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) } + } + + withSQLConf((HiveUtils.HIVE_THRIFT_SERVER_ASYNC.key, "true")) { + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + val opHandle = client.executeStatementAsync(sessionHandle, sql, confOverlay) + var status = client.getOperationStatus(opHandle) + while (!status.getState.isTerminal) { + Thread.sleep(10) + status = client.getOperationStatus(opHandle) + } + val e = status.getOperationException - assert(e.getMessage - .contains("The second argument of 'date_sub' function needs to be an integer.")) - assert(!e.getMessage.contains("" + - "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(e.getMessage + .contains("java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } } - withJdbcStatement { statement => - val e = intercept[SQLException] { - statement.executeQuery(sql) + Seq("true", "false").foreach { value => + withSQLConf((HiveUtils.HIVE_THRIFT_SERVER_ASYNC.key, value)) { + withJdbcStatement { statement => + val e = intercept[SQLException] { + statement.executeQuery(sql) + } + assert(e.getMessage.contains( + "The second argument of 'date_sub' function needs to be an integer.")) + assert(e.getMessage.contains( + "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } } - assert(e.getMessage - .contains("The second argument of 'date_sub' function needs to be an integer.")) - assert(e.getMessage.contains("" + - "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index 7f731f3d05e51..d0b829c240327 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -22,8 +22,10 @@ import scala.util.Random import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/sql/hive-thriftserver/v1.2/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala b/sql/hive-thriftserver/v1.2/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala new file mode 100644 index 0000000000000..1e684fa65b284 --- /dev/null +++ b/sql/hive-thriftserver/v1.2/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala @@ -0,0 +1,50 @@ +/* + * 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.hive.thriftserver + +import java.nio.ByteBuffer +import java.util.UUID + +import org.apache.hive.service.cli.OperationHandle +import org.apache.hive.service.cli.operation.GetCatalogsOperation +import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.cli.thrift.{THandleIdentifier, TOperationHandle, TOperationType} + +class GetCatalogsOperationMock(parentSession: HiveSession) + extends GetCatalogsOperation(parentSession) { + + override def runInternal(): Unit = {} + + override def getHandle: OperationHandle = { + val uuid: UUID = UUID.randomUUID() + val tHandleIdentifier: THandleIdentifier = new THandleIdentifier() + tHandleIdentifier.setGuid(getByteBufferFromUUID(uuid)) + tHandleIdentifier.setSecret(getByteBufferFromUUID(uuid)) + val tOperationHandle: TOperationHandle = new TOperationHandle() + tOperationHandle.setOperationId(tHandleIdentifier) + tOperationHandle.setOperationType(TOperationType.GET_TYPE_INFO) + tOperationHandle.setHasResultSetIsSet(false) + new OperationHandle(tOperationHandle) + } + + private def getByteBufferFromUUID(uuid: UUID): Array[Byte] = { + val bb: ByteBuffer = ByteBuffer.wrap(new Array[Byte](16)) + bb.putLong(uuid.getMostSignificantBits) + bb.putLong(uuid.getLeastSignificantBits) + bb.array + } +} diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 914d6d3612596..ea9ed57410045 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -648,7 +648,7 @@ public TGetPrimaryKeysResp GetPrimaryKeys(TGetPrimaryKeysReq req) resp.setOperationHandle(opHandle.toTOperationHandle()); resp.setStatus(OK_STATUS); } catch (Exception e) { - LOG.warn("Error getting functions: ", e); + LOG.warn("Error getting primary keys: ", e); resp.setStatus(HiveSQLException.toTStatus(e)); } return resp; @@ -666,7 +666,7 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) resp.setOperationHandle(opHandle.toTOperationHandle()); resp.setStatus(OK_STATUS); } catch (Exception e) { - LOG.warn("Error getting functions: ", e); + LOG.warn("Error getting cross reference: ", e); resp.setStatus(HiveSQLException.toTStatus(e)); } return resp; diff --git a/sql/hive-thriftserver/v2.3/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala b/sql/hive-thriftserver/v2.3/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala new file mode 100644 index 0000000000000..1bc9aaf672c3b --- /dev/null +++ b/sql/hive-thriftserver/v2.3/src/test/scala/ org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala @@ -0,0 +1,50 @@ +/* + * 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.hive.thriftserver + +import java.nio.ByteBuffer +import java.util.UUID + +import org.apache.hive.service.cli.OperationHandle +import org.apache.hive.service.cli.operation.GetCatalogsOperation +import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType} + +class GetCatalogsOperationMock(parentSession: HiveSession) + extends GetCatalogsOperation(parentSession) { + + override def runInternal(): Unit = {} + + override def getHandle: OperationHandle = { + val uuid: UUID = UUID.randomUUID() + val tHandleIdentifier: THandleIdentifier = new THandleIdentifier() + tHandleIdentifier.setGuid(getByteBufferFromUUID(uuid)) + tHandleIdentifier.setSecret(getByteBufferFromUUID(uuid)) + val tOperationHandle: TOperationHandle = new TOperationHandle() + tOperationHandle.setOperationId(tHandleIdentifier) + tOperationHandle.setOperationType(TOperationType.GET_TYPE_INFO) + tOperationHandle.setHasResultSetIsSet(false) + new OperationHandle(tOperationHandle) + } + + private def getByteBufferFromUUID(uuid: UUID): Array[Byte] = { + val bb: ByteBuffer = ByteBuffer.wrap(new Array[Byte](16)) + bb.putLong(uuid.getMostSignificantBits) + bb.putLong(uuid.getLeastSignificantBits) + bb.array + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala index c4885f2842597..f6aff10cbc147 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala @@ -54,9 +54,8 @@ private[sql] class PruneHiveTablePartitions(session: SparkSession) val normalizedFilters = DataSourceStrategy.normalizeExprs( filters.filter(f => f.deterministic && !SubqueryExpression.hasSubquery(f)), relation.output) val partitionColumnSet = AttributeSet(relation.partitionCols) - ExpressionSet(normalizedFilters.filter { f => - !f.references.isEmpty && f.references.subsetOf(partitionColumnSet) - }) + ExpressionSet( + normalizedFilters.flatMap(extractPredicatesWithinOutputSet(_, partitionColumnSet))) } /** @@ -103,9 +102,7 @@ private[sql] class PruneHiveTablePartitions(session: SparkSession) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case op @ PhysicalOperation(projections, filters, relation: HiveTableRelation) if filters.nonEmpty && relation.isPartitioned && relation.prunedPartitions.isEmpty => - val predicates = CNFWithGroupExpressionsByReference(filters.reduceLeft(And)) - val finalPredicates = if (predicates.nonEmpty) predicates else filters - val partitionKeyFilters = getPartitionKeyFilters(finalPredicates, relation) + val partitionKeyFilters = getPartitionKeyFilters(filters, relation) if (partitionKeyFilters.nonEmpty) { val newPartitions = prunePartitions(relation, partitionKeyFilters) val newTableMeta = updateTableMeta(relation.tableMeta, newPartitions) 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 3a7e92ee1c00b..6feaaea3dfb89 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 @@ -23,8 +23,9 @@ import scala.util.Properties import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.fs.Path -import org.scalatest.{BeforeAndAfterEach, Matchers} import org.scalatest.Assertions._ +import org.scalatest.BeforeAndAfterEach +import org.scalatest.matchers.must.Matchers import org.apache.spark._ import org.apache.spark.internal.Logging diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 2e4c01830432f..7f198632a1cd6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -371,8 +371,8 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && - (!hiveQuery.logical.isInstanceOf[ShowFunctionsStatement]) && - (!hiveQuery.logical.isInstanceOf[DescribeFunctionStatement]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && 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 327e4104d59a8..72aeb4f502213 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 @@ -20,11 +20,9 @@ package org.apache.spark.sql.hive.execution import scala.util.Random import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax -import org.scalatest.Matchers._ +import org.scalatest.matchers.must.Matchers._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 24aecb0274ece..ab37645b1c90c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.scalatest.Matchers._ +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala index c29e889c3a941..06aea084330fa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala @@ -55,6 +55,31 @@ class PruneHiveTablePartitionsSuite extends PrunePartitionSuiteBase { } } + test("Avoid generating too many predicates in partition pruning") { + withTempView("temp") { + withTable("t") { + sql( + s""" + |CREATE TABLE t(i INT, p0 INT, p1 INT) + |USING $format + |PARTITIONED BY (p0, p1)""".stripMargin) + + spark.range(0, 10, 1).selectExpr("id as col") + .createOrReplaceTempView("temp") + + for (part <- (0 to 25)) { + sql( + s""" + |INSERT OVERWRITE TABLE t PARTITION (p0='$part', p1='$part') + |SELECT col FROM temp""".stripMargin) + } + val scale = 20 + val predicate = (1 to scale).map(i => s"(p0 = '$i' AND p1 = '$i')").mkString(" OR ") + assertPrunedPartitions(s"SELECT * FROM t WHERE $predicate", scale) + } + } + } + override def getScanExecPartitionSize(plan: SparkPlan): Long = { plan.collectFirst { case p: HiveTableScanExec => p diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 920f6385f8e19..62a411a56159b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -92,7 +92,8 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi test("script") { withTempView("script_table") { assume(TestUtils.testCommandAvailable("/bin/bash")) - assume(TestUtils.testCommandAvailable("echo | sed")) + assume(TestUtils.testCommandAvailable("echo")) + assume(TestUtils.testCommandAvailable("sed")) val scriptFilePath = getTestResourcePath("test_script.sh") val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") df.createOrReplaceTempView("script_table") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala index 990d9425fb7fc..12ee5bea7c2f9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala @@ -288,4 +288,32 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { } } } + + test("SPARK-32234 read ORC table with column names all starting with '_col'") { + Seq("native", "hive").foreach { orcImpl => + Seq("false", "true").foreach { vectorized => + withSQLConf( + SQLConf.ORC_IMPLEMENTATION.key -> orcImpl, + SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized) { + withTable("test_hive_orc_impl") { + spark.sql( + s""" + | CREATE TABLE test_hive_orc_impl + | (_col1 INT, _col2 STRING, _col3 INT) + | STORED AS ORC + """.stripMargin) + spark.sql( + s""" + | INSERT INTO + | test_hive_orc_impl + | VALUES(9, '12', 2020) + """.stripMargin) + + val df = spark.sql("SELECT _col2 FROM test_hive_orc_impl") + checkAnswer(df, Row("12")) + } + } + } + } + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 0976494b6d094..86c20f5a46b9a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -26,8 +26,10 @@ import scala.concurrent.duration._ import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager @@ -91,7 +93,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), + new LiveListenerBus(conf), None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 368411cc2214b..420f7f20ce186 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -28,8 +28,10 @@ import scala.util.Random import org.apache.hadoop.conf.Configuration import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{doThrow, reset, spy} -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.Logging diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 679c58dbae92b..b54d60aa29c4f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -25,8 +25,9 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future import org.mockito.Mockito.{mock, reset, verifyNoMoreInteractions} -import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 7041e46f99b18..f797101992573 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -21,8 +21,10 @@ import scala.collection.mutable.Queue import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver -import org.scalatest._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.scalatestplus.selenium.WebBrowser diff --git a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala index 4c0dd0d258c53..cd867aa8132bc 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala @@ -23,9 +23,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.scalatest.BeforeAndAfter -import org.scalatest.Matchers._ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala index 1a0460cd669af..bad5cd2564988 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.streaming.scheduler.rate import scala.util.Random import org.scalatest.Inspectors.forAll -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.streaming.Seconds diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala index 10f92f9386173..01177f1cca70a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.streaming.ui import java.util.Properties -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.scheduler.SparkListenerJobStart import org.apache.spark.streaming.{LocalStreamingContext, _} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala index 576083723f8bd..7f96bbfd7b19a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala @@ -20,12 +20,13 @@ package org.apache.spark.streaming.ui import java.util.TimeZone import java.util.concurrent.TimeUnit -import org.scalatest.Matchers +import org.scalatest.matchers.must.Matchers +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.ui.{UIUtils => SparkUIUtils} -class UIUtilsSuite extends SparkFunSuite with Matchers{ +class UIUtilsSuite extends SparkFunSuite with Matchers { test("shortTimeUnitString") { assert("ns" === UIUtils.shortTimeUnitString(TimeUnit.NANOSECONDS))