diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index f71a7e4c7336..a50f4ce76861 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -37,6 +37,8 @@ on: - cron: '0 13 * * *' # Java 17 - cron: '0 16 * * *' + # branch-3.3 + - cron: '0 19 * * *' workflow_call: inputs: ansi_enabled: @@ -53,75 +55,108 @@ jobs: # Run all jobs for Apache Spark repository # Run only non-scheduled jobs for forked repositories if: github.repository == 'apache/spark' || github.event_name != 'schedule' + env: + GITHUB_PREV_SHA: ${{ github.event.before }} outputs: - java: ${{ steps.set-outputs.outputs.java }} - branch: ${{ steps.set-outputs.outputs.branch }} - hadoop: ${{ steps.set-outputs.outputs.hadoop }} - type: ${{ steps.set-outputs.outputs.type }} - envs: ${{ steps.set-outputs.outputs.envs }} + java: ${{ steps.config.outputs.java }} + branch: ${{ steps.config.outputs.branch }} + hadoop: ${{ steps.config.outputs.hadoop }} + type: ${{ steps.config.outputs.type }} + envs: ${{ steps.config.outputs.envs }} + + # Run scheduled jobs for Apache Spark only + # Run regular jobs for commit in both Apache Spark and forked repository, but only if changes exist + spark-required: >- + ${{ steps.config.outputs.type == 'scheduled' + || (steps.config.outputs.type == 'regular' && steps.changes.outputs.build-required == 'true') }} + + # Run PySpark coverage scheduled jobs for Apache Spark only + # Run scheduled jobs with JDK 17 in Apache Spark + # Run regular jobs for commit in both Apache Spark and forked repository, but only if pyspark changes exist + pyspark-required: >- + ${{ steps.config.outputs.type == 'pyspark-coverage-scheduled' + || (steps.config.outputs.type == 'scheduled' && steps.config.outputs.java == '17') + || (steps.config.outputs.type == 'regular' && steps.changes.outputs.pyspark-required == 'true') }} + + # Run scheduled jobs with JDK 17 in Apache Spark + # Run regular jobs for commit in both Apache Spark and forked repository, but only if sparkr changes exist + sparkr-required: >- + ${{ (steps.config.outputs.type == 'scheduled' && steps.config.outputs.java == '17') + || (steps.config.outputs.type == 'regular' && steps.changes.outputs.sparkr-required == 'true') }} + + # Run for regular jobs + lint-required: >- + ${{ steps.config.outputs.type == 'regular' }} + + # Run regular jobs for commit in both Apache Spark and forked repository, but only if changes exist + java-required: >- + ${{ steps.config.outputs.type == 'regular' && steps.changes.outputs.build-required == 'true' }} + + # Run regular jobs for commit in both Apache Spark and forked repository, but only if changes exist + scala-required: >- + ${{ steps.config.outputs.type == 'regular' && steps.changes.outputs.build-required == 'true' }} + + # Run regular jobs for commit in both Apache Spark and forked repository, but only if tpcds changes exist + tpcds-required: >- + ${{ steps.config.outputs.type == 'regular' && steps.changes.outputs.tpcds-required == 'true' }} + + # Run regular jobs for commit in both Apache Spark and forked repository, but only if docker changes exist + docker-required: >- + ${{ steps.config.outputs.type == 'regular' && steps.changes.outputs.docker-required == 'true' }} + steps: - name: Configure branch and additional environment variables - id: set-outputs + id: config run: | + # default values + java=8 + branch=master + type=scheduled + hadoop=hadoop3 + + # override default values based on event if [ "${{ github.event.schedule }}" = "0 1 * * *" ]; then - echo '::set-output name=java::8' - echo '::set-output name=branch::master' - echo '::set-output name=type::scheduled' - echo '::set-output name=envs::{}' - echo '::set-output name=hadoop::hadoop2' + envs='{}' + hadoop='hadoop2' elif [ "${{ github.event.schedule }}" = "0 4 * * *" ]; then - echo '::set-output name=java::8' - echo '::set-output name=branch::master' - echo '::set-output name=type::scheduled' - echo '::set-output name=envs::{"SCALA_PROFILE": "scala2.13"}' - echo '::set-output name=hadoop::hadoop3' + envs='{"SCALA_PROFILE": "scala2.13"}' + hadoop='hadoop3' elif [ "${{ github.event.schedule }}" = "0 7 * * *" ]; then - echo '::set-output name=java::8' - echo '::set-output name=branch::branch-3.2' - echo '::set-output name=type::scheduled' - echo '::set-output name=envs::{"SCALA_PROFILE": "scala2.13"}' - echo '::set-output name=hadoop::hadoop3.2' + branch='branch-3.2' + envs='{"SCALA_PROFILE": "scala2.13"}' + hadoop='hadoop3.2' elif [ "${{ github.event.schedule }}" = "0 10 * * *" ]; then - echo '::set-output name=java::8' - echo '::set-output name=branch::master' - echo '::set-output name=type::pyspark-coverage-scheduled' - echo '::set-output name=envs::{"PYSPARK_CODECOV": "true"}' - echo '::set-output name=hadoop::hadoop3' + type=pyspark-coverage-scheduled + envs='{"PYSPARK_CODECOV": "true"}' elif [ "${{ github.event.schedule }}" = "0 13 * * *" ]; then - echo '::set-output name=java::11' - echo '::set-output name=branch::master' - echo '::set-output name=type::scheduled' - echo '::set-output name=envs::{"SKIP_MIMA": "true", "SKIP_UNIDOC": "true"}' - echo '::set-output name=hadoop::hadoop3' + java=11 + envs='{"SKIP_MIMA": "true", "SKIP_UNIDOC": "true"}' elif [ "${{ github.event.schedule }}" = "0 16 * * *" ]; then - echo '::set-output name=java::17' - echo '::set-output name=branch::master' - echo '::set-output name=type::scheduled' - echo '::set-output name=envs::{"SKIP_MIMA": "true", "SKIP_UNIDOC": "true"}' - echo '::set-output name=hadoop::hadoop3' + java=17 + envs='{"SKIP_MIMA": "true", "SKIP_UNIDOC": "true"}' + elif [ "${{ github.event.schedule }}" = "0 19 * * *" ]; then + branch='branch-3.3' + envs='{"SCALA_PROFILE": "scala2.13"}' else - echo '::set-output name=java::8' - echo '::set-output name=branch::master' # NOTE: UPDATE THIS WHEN CUTTING BRANCH - echo '::set-output name=type::regular' - echo '::set-output name=envs::{"SPARK_ANSI_SQL_MODE": "${{ inputs.ansi_enabled }}"}' - echo '::set-output name=hadoop::hadoop3' + # NOTE: UPDATE THIS WHEN CUTTING BRANCH + branch=master + type=regular + envs='{"SPARK_ANSI_SQL_MODE": "${{ inputs.ansi_enabled }}"}' fi - precondition: - name: Check changes - runs-on: ubuntu-20.04 - needs: configure-jobs - env: - GITHUB_PREV_SHA: ${{ github.event.before }} - outputs: - required: ${{ steps.set-outputs.outputs.required }} - steps: + # output values + echo "::set-output name=java::$java" + echo "::set-output name=branch::$branch" + echo "::set-output name=type::$type" + echo "::set-output name=envs::$envs" + echo "::set-output name=hadoop::$hadoop" + - name: Checkout Spark repository uses: actions/checkout@v2 with: fetch-depth: 0 repository: apache/spark - ref: ${{ needs.configure-jobs.outputs.branch }} + ref: ${{ steps.config.outputs.branch }} - name: Sync the current branch with the latest in Apache Spark if: github.repository != 'apache/spark' run: | @@ -129,27 +164,31 @@ jobs: git fetch https://github.com/$GITHUB_REPOSITORY.git ${GITHUB_REF#refs/heads/} git -c user.name='Apache Spark Test Account' -c user.email='sparktestacc@gmail.com' merge --no-commit --progress --squash FETCH_HEAD git -c user.name='Apache Spark Test Account' -c user.email='sparktestacc@gmail.com' commit -m "Merged commit" --allow-empty + - name: Check all modules - id: set-outputs + id: changes run: | - build=`./dev/is-changed.py -m avro,build,catalyst,core,docker-integration-tests,examples,graphx,hadoop-cloud,hive,hive-thriftserver,kubernetes,kvstore,launcher,mesos,mllib,mllib-local,network-common,network-shuffle,pyspark-core,pyspark-ml,pyspark-mllib,pyspark-pandas,pyspark-pandas-slow,pyspark-resource,pyspark-sql,pyspark-streaming,repl,sketch,spark-ganglia-lgpl,sparkr,sql,sql-kafka-0-10,streaming,streaming-kafka-0-10,streaming-kinesis-asl,tags,unsafe,yarn` - pyspark=`./dev/is-changed.py -m avro,build,catalyst,core,graphx,hive,kvstore,launcher,mllib,mllib-local,network-common,network-shuffle,pyspark-core,pyspark-ml,pyspark-mllib,pyspark-pandas,pyspark-pandas-slow,pyspark-resource,pyspark-sql,pyspark-streaming,repl,sketch,sql,tags,unsafe` - sparkr=`./dev/is-changed.py -m avro,build,catalyst,core,hive,kvstore,launcher,mllib,mllib-local,network-common,network-shuffle,repl,sketch,sparkr,sql,tags,unsafe` - tpcds=`./dev/is-changed.py -m build,catalyst,core,hive,kvstore,launcher,network-common,network-shuffle,repl,sketch,sql,tags,unsafe` - docker=`./dev/is-changed.py -m build,catalyst,core,docker-integration-tests,hive,kvstore,launcher,network-common,network-shuffle,repl,sketch,sql,tags,unsafe` - echo "{\"build\": \"$build\", \"pyspark\": \"$pyspark\", \"sparkr\": \"$sparkr\", \"tpcds\": \"$tpcds\", \"docker\": \"$docker\"}" > required.json - cat required.json - echo "::set-output name=required::$(cat required.json)" + # is-changed.py is missing in branch-3.2, and it might run in scheduled build, see also SPARK-39517 + build=true; pyspark=true; sparkr=true; tpcds=true; docker=true; + if [ -f "./dev/is-changed.py" ]; then + build=`./dev/is-changed.py -m avro,build,catalyst,core,docker-integration-tests,examples,graphx,hadoop-cloud,hive,hive-thriftserver,kubernetes,kvstore,launcher,mesos,mllib,mllib-local,network-common,network-shuffle,pyspark-core,pyspark-ml,pyspark-mllib,pyspark-pandas,pyspark-pandas-slow,pyspark-resource,pyspark-sql,pyspark-streaming,repl,sketch,spark-ganglia-lgpl,sparkr,sql,sql-kafka-0-10,streaming,streaming-kafka-0-10,streaming-kinesis-asl,tags,unsafe,yarn` + pyspark=`./dev/is-changed.py -m avro,build,catalyst,core,graphx,hive,kvstore,launcher,mllib,mllib-local,network-common,network-shuffle,pyspark-core,pyspark-ml,pyspark-mllib,pyspark-pandas,pyspark-pandas-slow,pyspark-resource,pyspark-sql,pyspark-streaming,repl,sketch,sql,tags,unsafe` + sparkr=`./dev/is-changed.py -m avro,build,catalyst,core,hive,kvstore,launcher,mllib,mllib-local,network-common,network-shuffle,repl,sketch,sparkr,sql,tags,unsafe` + tpcds=`./dev/is-changed.py -m build,catalyst,core,hive,kvstore,launcher,network-common,network-shuffle,repl,sketch,sql,tags,unsafe` + docker=`./dev/is-changed.py -m build,catalyst,core,docker-integration-tests,hive,kvstore,launcher,network-common,network-shuffle,repl,sketch,sql,tags,unsafe` + fi - # Build: build Spark and run the tests for specified modules. - build: + echo "::set-output name=build-required::$build" + echo "::set-output name=pyspark-required::$pyspark" + echo "::set-output name=sparkr-required::$sparkr" + echo "::set-output name=tpcds-required::$tpcds" + echo "::set-output name=docker-required::$docker" + + # Build Spark and run the tests for specified modules. + spark: name: "Build modules (${{ format('{0}, {1} job', needs.configure-jobs.outputs.branch, needs.configure-jobs.outputs.type) }}): ${{ matrix.modules }} ${{ matrix.comment }} (JDK ${{ matrix.java }}, ${{ matrix.hadoop }}, ${{ matrix.hive }})" - needs: [configure-jobs, precondition] - # Run scheduled jobs for Apache Spark only - # Run regular jobs for commit in both Apache Spark and forked repository, but only if changes exist - if: >- - needs.configure-jobs.outputs.type == 'scheduled' - || (needs.configure-jobs.outputs.type == 'regular' && fromJson(needs.precondition.outputs.required).build == 'true') + needs: [configure-jobs] + if: needs.configure-jobs.outputs.spark-required == 'true' # Ubuntu 20.04 is the latest LTS. The next LTS is 22.04. runs-on: ubuntu-20.04 strategy: @@ -287,16 +326,11 @@ jobs: name: unit-tests-log-${{ matrix.modules }}-${{ matrix.comment }}-${{ matrix.java }}-${{ matrix.hadoop }}-${{ matrix.hive }} path: "**/target/unit-tests.log" + # Build PySpark and run the tests for specified modules. pyspark: - needs: [configure-jobs, precondition] - # Run PySpark coverage scheduled jobs for Apache Spark only - # Run scheduled jobs with JDK 17 in Apache Spark - # Run regular jobs for commit in both Apache Spark and forked repository, but only if pyspark changes exist - if: >- - needs.configure-jobs.outputs.type == 'pyspark-coverage-scheduled' - || (needs.configure-jobs.outputs.type == 'scheduled' && needs.configure-jobs.outputs.java == '17') - || (needs.configure-jobs.outputs.type == 'regular' && fromJson(needs.precondition.outputs.required).pyspark == 'true') name: "Build modules (${{ format('{0}, {1} job', needs.configure-jobs.outputs.branch, needs.configure-jobs.outputs.type) }}): ${{ matrix.modules }}" + needs: [configure-jobs] + if: needs.configure-jobs.outputs.pyspark-required == 'true' runs-on: ubuntu-20.04 container: image: dongjoon/apache-spark-github-action-image:20220207 @@ -397,13 +431,9 @@ jobs: path: "**/target/unit-tests.log" sparkr: - needs: [configure-jobs, precondition] - # Run scheduled jobs with JDK 17 in Apache Spark - # Run regular jobs for commit in both Apache Spark and forked repository, but only if sparkr changes exist - if: >- - (needs.configure-jobs.outputs.type == 'scheduled' && needs.configure-jobs.outputs.java == '17') - || (needs.configure-jobs.outputs.type == 'regular' && fromJson(needs.precondition.outputs.required).sparkr == 'true') name: "Build modules: sparkr" + needs: [configure-jobs] + if: needs.configure-jobs.outputs.sparkr-required == 'true' runs-on: ubuntu-20.04 container: image: dongjoon/apache-spark-github-action-image:20220207 @@ -468,9 +498,9 @@ jobs: # Static analysis, and documentation build lint: - needs: configure-jobs - if: needs.configure-jobs.outputs.type == 'regular' - name: Linters, licenses, dependencies and documentation generation + name: "Linters, licenses, dependencies and documentation generation" + needs: [configure-jobs] + if: needs.configure-jobs.outputs.lint-required == 'true' runs-on: ubuntu-20.04 env: LC_ALL: C.UTF-8 @@ -583,17 +613,16 @@ jobs: bundle exec jekyll build java-11-17: - needs: [configure-jobs, precondition] - # Run regular jobs for commit in both Apache Spark and forked repository, but only if changes exist - if: needs.configure-jobs.outputs.type == 'regular' && fromJson(needs.precondition.outputs.required).build == 'true' - name: Java ${{ matrix.java }} build with Maven + name: "Java ${{ matrix.java }} build with Maven" + needs: [configure-jobs] + if: needs.configure-jobs.outputs.java-required == 'true' + runs-on: ubuntu-20.04 strategy: fail-fast: false matrix: java: - 11 - 17 - runs-on: ubuntu-20.04 steps: - name: Checkout Spark repository uses: actions/checkout@v2 @@ -639,10 +668,9 @@ jobs: rm -rf ~/.m2/repository/org/apache/spark scala-213: - needs: [configure-jobs, precondition] - # Run regular jobs for commit in both Apache Spark and forked repository, but only if changes exist - if: needs.configure-jobs.outputs.type == 'regular' && fromJson(needs.precondition.outputs.required).build == 'true' - name: Scala 2.13 build with SBT + name: "Scala 2.13 build with SBT" + needs: [configure-jobs] + if: needs.configure-jobs.outputs.scala-required == 'true' runs-on: ubuntu-20.04 steps: - name: Checkout Spark repository @@ -685,10 +713,9 @@ jobs: ./build/sbt -Pyarn -Pmesos -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pdocker-integration-tests -Pkubernetes-integration-tests -Pspark-ganglia-lgpl -Pscala-2.13 compile test:compile tpcds-1g: - needs: [configure-jobs, precondition] - # Run regular jobs for commit in both Apache Spark and forked repository, but only if tpcds changes exist - if: needs.configure-jobs.outputs.type == 'regular' && fromJson(needs.precondition.outputs.required).tpcds == 'true' - name: Run TPC-DS queries with SF=1 + name: "Run TPC-DS queries with SF=1" + needs: [configure-jobs] + if: needs.configure-jobs.outputs.tpcds-required == 'true' runs-on: ubuntu-20.04 env: SPARK_LOCAL_IP: localhost @@ -781,10 +808,9 @@ jobs: path: "**/target/unit-tests.log" docker-integration-tests: - needs: [configure-jobs, precondition] - # Run regular jobs for commit in both Apache Spark and forked repository, but only if docker changes exist - if: needs.configure-jobs.outputs.type == 'regular' && fromJson(needs.precondition.outputs.required).docker == 'true' - name: Run Docker integration tests + name: "Run Docker integration tests" + needs: [configure-jobs] + if: needs.configure-jobs.outputs.docker-required == 'true' runs-on: ubuntu-20.04 env: HADOOP_PROFILE: ${{ needs.configure-jobs.outputs.hadoop }} diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 275566589f54..d4c0910c5ad7 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -251,12 +251,6 @@ "Key does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set to \"false\" to bypass this error." ] }, - "MISSING_COLUMN" : { - "message" : [ - "Column '' does not exist. Did you mean one of the following? []" - ], - "sqlState" : "42000" - }, "MISSING_STATIC_PARTITION_COLUMN" : { "message" : [ "Unknown static partition column: " @@ -352,6 +346,12 @@ ], "sqlState" : "42000" }, + "UNRESOLVED_COLUMN" : { + "message" : [ + "A column or function parameter with name cannot be resolved. Did you mean one of the following? []" + ], + "sqlState" : "42000" + }, "UNSUPPORTED_DATATYPE" : { "message" : [ "Unsupported data type " diff --git a/core/src/main/resources/org/apache/spark/ui/static/log-view.js b/core/src/main/resources/org/apache/spark/ui/static/log-view.js index 2f416d8210e1..92df087f4bd9 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/log-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/log-view.js @@ -85,7 +85,7 @@ function loadMore() { if (retStartByte == 0) { disableMoreButton(); } - $("pre", ".log-content").prepend(cleanData); + $("pre", ".log-content").prepend(document.createTextNode(cleanData)); curLogLength = curLogLength + (startByte - retStartByte); startByte = retStartByte; @@ -115,7 +115,7 @@ function loadNew() { var retLogLength = dataInfo[2]; var cleanData = data.substring(newlineIndex + 1); - $("pre", ".log-content").append(cleanData); + $("pre", ".log-content").append(document.createTextNode(cleanData)); curLogLength = curLogLength + (retEndByte - retStartByte); endByte = retEndByte; diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 2beca6fddb27..69a74146fad1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -77,7 +77,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String @GuardedBy("self") private var daemon: Process = null - val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) + val daemonHost = InetAddress.getLoopbackAddress() @GuardedBy("self") private var daemonPort: Int = 0 @GuardedBy("self") @@ -153,7 +153,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String private def createSimpleWorker(): (Socket, Option[Int]) = { var serverSocket: ServerSocket = null try { - serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) + serverSocket = new ServerSocket(0, 1, InetAddress.getLoopbackAddress()) // Create and start the worker val pb = new ProcessBuilder(Arrays.asList(pythonExec, "-m", workerModule)) @@ -164,6 +164,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String workerEnv.put("PYTHONUNBUFFERED", "YES") workerEnv.put("PYTHON_WORKER_FACTORY_PORT", serverSocket.getLocalPort.toString) workerEnv.put("PYTHON_WORKER_FACTORY_SECRET", authHelper.secret) + if (Utils.preferIPv6) { + workerEnv.put("SPARK_PREFER_IPV6", "True") + } val worker = pb.start() // Redirect worker stdout and stderr @@ -211,6 +214,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String workerEnv.putAll(envVars.asJava) workerEnv.put("PYTHONPATH", pythonPath) workerEnv.put("PYTHON_WORKER_FACTORY_SECRET", authHelper.secret) + if (Utils.preferIPv6) { + workerEnv.put("SPARK_PREFER_IPV6", "True") + } // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u: workerEnv.put("PYTHONUNBUFFERED", "YES") daemon = pb.start() diff --git a/core/src/main/scala/org/apache/spark/security/SocketAuthServer.scala b/core/src/main/scala/org/apache/spark/security/SocketAuthServer.scala index 35990b5a5928..1c6cae8f10e7 100644 --- a/core/src/main/scala/org/apache/spark/security/SocketAuthServer.scala +++ b/core/src/main/scala/org/apache/spark/security/SocketAuthServer.scala @@ -49,7 +49,7 @@ private[spark] abstract class SocketAuthServer[T]( private def startServer(): (Int, String) = { logTrace("Creating listening socket") - val serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) + val serverSocket = new ServerSocket(0, 1, InetAddress.getLoopbackAddress()) // Close the socket if no connection in the configured seconds val timeout = authHelper.conf.get(PYTHON_AUTH_SOCKET_TIMEOUT).toInt logTrace(s"Setting timeout to $timeout sec") diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index cf93897f97d8..56392f0239c8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1988,6 +1988,11 @@ private[spark] object Utils extends Logging { */ val isMacOnAppleSilicon = SystemUtils.IS_OS_MAC_OSX && SystemUtils.OS_ARCH.equals("aarch64") + /** + * Whether the underlying JVM prefer IPv6 addresses. + */ + val preferIPv6 = "true".equals(System.getProperty("java.net.preferIPv6Addresses")) + /** * Pattern for matching a Windows drive, which contains only a single alphabet character. */ diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index 91019973dea3..4b14e2402a73 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -160,7 +160,7 @@ class SparkThrowableSuite extends SparkFunSuite { test("Check if message parameters match message format") { // Requires 2 args intercept[IllegalFormatException] { - getMessage("MISSING_COLUMN", null, Array.empty) + getMessage("UNRESOLVED_COLUMN", null, Array.empty) } // Does not fail with too many args (expects 0 args) @@ -172,8 +172,9 @@ class SparkThrowableSuite extends SparkFunSuite { } test("Error message is formatted") { - assert(getMessage("MISSING_COLUMN", null, Array("foo", "bar, baz")) == - "[MISSING_COLUMN] Column 'foo' does not exist. Did you mean one of the following? [bar, baz]") + assert(getMessage("UNRESOLVED_COLUMN", null, Array("`foo`", "`bar`, `baz`")) == + "[UNRESOLVED_COLUMN] A column or function parameter with name `foo` cannot be resolved. " + + "Did you mean one of the following? [`bar`, `baz`]") } test("Try catching legacy SparkError") { diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 894588534792..c118c58cafea 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -32,7 +32,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate -import org.apache.spark.util.Utils +import org.apache.spark.util.{DependencyUtils, Utils} class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { @@ -304,4 +304,10 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { s" Resolved jars are: $jarPath") } } + + test("SPARK-39501: Resolve maven dependenicy in IPv6") { + assume(Utils.preferIPv6) + DependencyUtils.resolveMavenDependencies( + URI.create("ivy://org.apache.logging.log4j:log4j-api:2.17.2")) + } } 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 1aa846b3ac43..6322661f4afd 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 @@ -74,7 +74,7 @@ abstract class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with private var provider: FsHistoryProvider = null private var server: HistoryServer = null - private val localhost: String = Utils.localCanonicalHostName() + private val localhost: String = Utils.localHostNameForURI() private var port: Int = -1 protected def diskBackend: HybridStoreDiskBackend.Value 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 1fac3522aed5..b66b39c3c077 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 @@ -325,7 +325,7 @@ class MasterSuite extends SparkFunSuite val conf = new SparkConf() val localCluster = LocalSparkCluster(2, 2, 512, conf) localCluster.start() - val masterUrl = s"http://${Utils.localCanonicalHostName()}:${localCluster.masterWebUIPort}" + val masterUrl = s"http://${Utils.localHostNameForURI()}:${localCluster.masterWebUIPort}" try { eventually(timeout(50.seconds), interval(100.milliseconds)) { val json = Utils @@ -362,7 +362,7 @@ class MasterSuite extends SparkFunSuite conf.set(UI_REVERSE_PROXY, true) val localCluster = LocalSparkCluster(2, 2, 512, conf) localCluster.start() - val masterUrl = s"http://${Utils.localCanonicalHostName()}:${localCluster.masterWebUIPort}" + val masterUrl = s"http://${Utils.localHostNameForURI()}:${localCluster.masterWebUIPort}" try { eventually(timeout(50.seconds), interval(100.milliseconds)) { val json = Utils @@ -400,7 +400,7 @@ class MasterSuite extends SparkFunSuite conf.set(UI_REVERSE_PROXY_URL, reverseProxyUrl) val localCluster = LocalSparkCluster(2, 2, 512, conf) localCluster.start() - val masterUrl = s"http://${Utils.localCanonicalHostName()}:${localCluster.masterWebUIPort}" + val masterUrl = s"http://${Utils.localHostNameForURI()}:${localCluster.masterWebUIPort}" try { eventually(timeout(50.seconds), interval(100.milliseconds)) { val json = Utils 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 253517fdcf94..b28651ea79cc 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 @@ -69,7 +69,7 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { when(master.idToApp).thenReturn(HashMap[String, ApplicationInfo]((activeApp.id, activeApp))) - val url = s"http://${Utils.localCanonicalHostName()}:${masterWebUI.boundPort}/app/kill/" + val url = s"http://${Utils.localHostNameForURI()}:${masterWebUI.boundPort}/app/kill/" val body = convPostDataToString(Map(("id", activeApp.id), ("terminate", "true"))) val conn = sendHttpRequest(url, "POST", body) conn.getResponseCode @@ -80,7 +80,7 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { test("kill driver") { val activeDriverId = "driver-0" - val url = s"http://${Utils.localCanonicalHostName()}:${masterWebUI.boundPort}/driver/kill/" + val url = s"http://${Utils.localHostNameForURI()}:${masterWebUI.boundPort}/driver/kill/" val body = convPostDataToString(Map(("id", activeDriverId), ("terminate", "true"))) val conn = sendHttpRequest(url, "POST", body) conn.getResponseCode @@ -90,7 +90,7 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { } private def testKillWorkers(hostnames: Seq[String]): Unit = { - val url = s"http://${Utils.localCanonicalHostName()}:${masterWebUI.boundPort}/workers/kill/" + val url = s"http://${Utils.localHostNameForURI()}:${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 @@ -100,7 +100,7 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { } test("Kill one host") { - testKillWorkers(Seq("${Utils.localCanonicalHostName()}")) + testKillWorkers(Seq(s"${Utils.localHostNameForURI()}")) } test("Kill multiple hosts") { 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 473782ee28d1..206a7d68553b 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -50,7 +50,8 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { .setSparkHome(sys.props("spark.test.home")) .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) .setConf(UI_ENABLED.key, "false") - .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, s"-Dtest.appender=console") + .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, + s"-Dtest.appender=console -Djava.net.preferIPv6Addresses=${Utils.preferIPv6}") .setMaster(master) .setAppResource(SparkLauncher.NO_RESOURCE) .setMainClass(TestApp.getClass.getName().stripSuffix("$")) diff --git a/dev/check-license b/dev/check-license index bd255954d6db..f1cd5a5f1d44 100755 --- a/dev/check-license +++ b/dev/check-license @@ -20,7 +20,7 @@ acquire_rat_jar () { - URL="https://repo.maven.apache.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" + URL="${DEFAULT_ARTIFACT_REPOSITORY:-https://repo1.maven.org/maven2/}org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" JAR="$rat_jar" diff --git a/dev/deps/spark-deps-hadoop-2-hive-2.3 b/dev/deps/spark-deps-hadoop-2-hive-2.3 index 010bccc9b68f..bcd773264fa6 100644 --- a/dev/deps/spark-deps-hadoop-2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2-hive-2.3 @@ -219,9 +219,9 @@ objenesis/3.2//objenesis-3.2.jar okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar -orc-core/1.7.4//orc-core-1.7.4.jar -orc-mapreduce/1.7.4//orc-mapreduce-1.7.4.jar -orc-shims/1.7.4//orc-shims-1.7.4.jar +orc-core/1.7.5//orc-core-1.7.5.jar +orc-mapreduce/1.7.5//orc-mapreduce-1.7.5.jar +orc-shims/1.7.5//orc-shims-1.7.5.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 7c5e66b91c72..53420957d750 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -208,9 +208,9 @@ opencsv/2.3//opencsv-2.3.jar opentracing-api/0.33.0//opentracing-api-0.33.0.jar opentracing-noop/0.33.0//opentracing-noop-0.33.0.jar opentracing-util/0.33.0//opentracing-util-0.33.0.jar -orc-core/1.7.4//orc-core-1.7.4.jar -orc-mapreduce/1.7.4//orc-mapreduce-1.7.4.jar -orc-shims/1.7.4//orc-shims-1.7.4.jar +orc-core/1.7.5//orc-core-1.7.5.jar +orc-mapreduce/1.7.5//orc-mapreduce-1.7.5.jar +orc-shims/1.7.5//orc-shims-1.7.5.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 487cb27b93fe..bfee3301f8e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -24,6 +24,7 @@ import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util.Identifiable import org.apache.spark.sql.{Dataset, Row, SparkSession} +import org.apache.spark.sql.functions.col /** * A simple example demonstrating how to write your own learning algorithm using Estimator, @@ -120,8 +121,10 @@ private class MyLogisticRegression(override val uid: String) // This method is used by fit() override protected def train(dataset: Dataset[_]): MyLogisticRegressionModel = { - // Extract columns from data using helper method. - val oldDataset = extractLabeledPoints(dataset) + // Extract columns from data. + val oldDataset = dataset.select(col($(labelCol)).cast("double"), col($(featuresCol))) + .rdd + .map { case Row(l: Double, f: Vector) => LabeledPoint(l, f) } // Do learning to estimate the coefficients vector. val numFeatures = oldDataset.take(1)(0).features.size diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index e0b128e36981..9c6eb880c80c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -18,14 +18,11 @@ package org.apache.spark.ml import org.apache.spark.annotation.Since -import org.apache.spark.ml.feature.{Instance, LabeledPoint} -import org.apache.spark.ml.functions.checkNonNegativeWeight -import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -63,40 +60,6 @@ private[ml] trait PredictorParams extends Params } SchemaUtils.appendColumn(schema, $(predictionCol), DoubleType) } - - /** - * Extract [[labelCol]], weightCol(if any) and [[featuresCol]] from the given dataset, - * and put it in an RDD with strong types. - */ - protected def extractInstances(dataset: Dataset[_]): RDD[Instance] = { - val w = this match { - case p: HasWeightCol => - if (isDefined(p.weightCol) && $(p.weightCol).nonEmpty) { - checkNonNegativeWeight((col($(p.weightCol)).cast(DoubleType))) - } else { - lit(1.0) - } - } - - dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } - } - - /** - * Extract [[labelCol]], weightCol(if any) and [[featuresCol]] from the given dataset, - * and put it in an RDD with strong types. - * Validate the output instances with the given function. - */ - protected def extractInstances( - dataset: Dataset[_], - validateInstance: Instance => Unit): RDD[Instance] = { - extractInstances(dataset).map { instance => - validateInstance(instance) - instance - } - } } /** @@ -176,16 +139,6 @@ abstract class Predictor[ override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema, fitting = true, featuresDataType) } - - /** - * Extract [[labelCol]] and [[featuresCol]] from the given dataset, - * and put it in an RDD with strong types. - */ - protected def extractLabeledPoints(dataset: Dataset[_]): RDD[LabeledPoint] = { - dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => LabeledPoint(label, features) - } - } } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 09324e2087d5..2d7719a29cab 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,17 +17,13 @@ package org.apache.spark.ml.classification -import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.DatasetUtils._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} @@ -44,23 +40,6 @@ private[spark] trait ClassifierParams val parentSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) SchemaUtils.appendColumn(parentSchema, $(rawPredictionCol), new VectorUDT) } - - /** - * Extract [[labelCol]], weightCol(if any) and [[featuresCol]] from the given dataset, - * and put it in an RDD with strong types. - * Validates the label on the classifier is a valid integer in the range [0, numClasses). - */ - protected def extractInstances( - dataset: Dataset[_], - numClasses: Int): RDD[Instance] = { - val validateInstance = (instance: Instance) => { - val label = instance.label - require(label.toLong == label && label >= 0 && label < numClasses, s"Classifier was given" + - s" dataset with invalid label $label. Labels must be integers in range" + - s" [0, $numClasses).") - } - extractInstances(dataset, validateInstance) - } } /** @@ -81,89 +60,6 @@ abstract class Classifier[ def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] // TODO: defaultEvaluator (follow-up PR) - - /** - * Extract [[labelCol]] and [[featuresCol]] from the given dataset, - * and put it in an RDD with strong types. - * - * @param dataset DataFrame with columns for labels ([[org.apache.spark.sql.types.NumericType]]) - * and features (`Vector`). - * @param numClasses Number of classes label can take. Labels must be integers in the range - * [0, numClasses). - * @note Throws `SparkException` if any label is a non-integer or is negative - */ - protected def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = { - validateNumClasses(numClasses) - dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - validateLabel(label, numClasses) - LabeledPoint(label, features) - } - } - - /** - * Validates that number of classes is greater than zero. - * - * @param numClasses Number of classes label can take. - */ - protected def validateNumClasses(numClasses: Int): Unit = { - require(numClasses > 0, s"Classifier (in extractLabeledPoints) found numClasses =" + - s" $numClasses, but requires numClasses > 0.") - } - - /** - * Validates the label on the classifier is a valid integer in the range [0, numClasses). - * - * @param label The label to validate. - * @param numClasses Number of classes label can take. Labels must be integers in the range - * [0, numClasses). - */ - protected def validateLabel(label: Double, numClasses: Int): Unit = { - require(label.toLong == label && label >= 0 && label < numClasses, s"Classifier was given" + - s" dataset with invalid label $label. Labels must be integers in range" + - s" [0, $numClasses).") - } - - /** - * Get the number of classes. This looks in column metadata first, and if that is missing, - * then this assumes classes are indexed 0,1,...,numClasses-1 and computes numClasses - * by finding the maximum label value. - * - * Label validation (ensuring all labels are integers >= 0) needs to be handled elsewhere, - * such as in `extractLabeledPoints()`. - * - * @param dataset Dataset which contains a column [[labelCol]] - * @param maxNumClasses Maximum number of classes allowed when inferred from data. If numClasses - * is specified in the metadata, then maxNumClasses is ignored. - * @return number of classes - * @throws IllegalArgumentException if metadata does not specify numClasses, and the - * actual numClasses exceeds maxNumClasses - */ - protected def getNumClasses(dataset: Dataset[_], maxNumClasses: Int = 100): Int = { - MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { - case Some(n: Int) => n - case None => - // Get number of classes from dataset itself. - val maxLabelRow: Array[Row] = dataset - .select(max(checkClassificationLabels($(labelCol), Some(maxNumClasses)))) - .take(1) - if (maxLabelRow.isEmpty || maxLabelRow(0).get(0) == null) { - throw new SparkException("ML algorithm was given empty dataset.") - } - val maxDoubleLabel: Double = maxLabelRow.head.getDouble(0) - require((maxDoubleLabel + 1).isValidInt, s"Classifier found max label value =" + - s" $maxDoubleLabel but requires integers in range [0, ... ${Int.MaxValue})") - val numClasses = maxDoubleLabel.toInt + 1 - require(numClasses <= maxNumClasses, s"Classifier inferred $numClasses from label values" + - s" in column $labelCol, but this exceeded the max numClasses ($maxNumClasses) allowed" + - s" to be inferred from values. To avoid this error for labels with > $maxNumClasses" + - s" classes, specify numClasses explicitly in the metadata; this can be done by applying" + - s" StringIndexer to the label column.") - logInfo(this.getClass.getCanonicalName + s" inferred $numClasses classes for" + - s" labelCol=$labelCol since numClasses was not specified in the column metadata.") - numClasses - } - } } /** 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 ec9e779709dd..688d2d18f483 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 @@ -117,14 +117,13 @@ class DecisionTreeClassifier @Since("1.4.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) val categoricalFeatures = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val numClasses = getNumClasses(dataset) + val numClasses = getNumClasses(dataset, $(labelCol)) if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + ".train() called with non-matching numClasses and thresholds.length." + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - validateNumClasses(numClasses) val instances = dataset.select( checkClassificationLabels($(labelCol), Some(numClasses)), 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 a2e6f0c49ee7..51f312cf1833 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 @@ -190,7 +190,7 @@ class FMClassifier @Since("3.0.0") ( miniBatchFraction, initStd, maxIter, stepSize, tol, solver, thresholds) instr.logNumClasses(numClasses) - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val numFeatures = getNumFeatures(dataset, $(featuresCol)) instr.logNumFeatures(numFeatures) val handlePersistence = dataset.storageLevel == StorageLevel.NONE 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 a767bc014451..3910beda3d0a 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 @@ -22,14 +22,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{BLAS, DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.DatasetUtils._ +import org.apache.spark.ml.util.DatasetUtils.extractInstances import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} @@ -169,21 +168,12 @@ class GBTClassifier @Since("1.4.0") ( override protected def train( dataset: Dataset[_]): GBTClassificationModel = instrumented { instr => - - def extractInstances(df: Dataset[_]) = { - df.select( - checkClassificationLabels($(labelCol), Some(2)), - checkNonNegativeWeights(get(weightCol)), - checkNonNanVectors($(featuresCol)) - ).rdd.map { case Row(l: Double, w: Double, v: Vector) => Instance(l, w, v) } - } - val withValidation = isDefined(validationIndicatorCol) && $(validationIndicatorCol).nonEmpty val (trainDataset, validationDataset) = if (withValidation) { - (extractInstances(dataset.filter(not(col($(validationIndicatorCol))))), - extractInstances(dataset.filter(col($(validationIndicatorCol))))) + (extractInstances(this, dataset.filter(not(col($(validationIndicatorCol)))), Some(2)), + extractInstances(this, dataset.filter(col($(validationIndicatorCol))), Some(2))) } else { - (extractInstances(dataset), null) + (extractInstances(this, dataset, Some(2)), null) } val numClasses = 2 @@ -390,7 +380,7 @@ class GBTClassificationModel private[ml]( */ @Since("2.4.0") def evaluateEachIteration(dataset: Dataset[_]): Array[Double] = { - val data = extractInstances(dataset) + val data = extractInstances(this, dataset, Some(2)) GradientBoostedTrees.evaluateEachIteration(data, trees, treeWeights, loss, OldAlgo.Classification) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 25f4e103ac73..048e5949e1c1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -141,7 +141,7 @@ class RandomForestClassifier @Since("1.4.0") ( instr.logDataset(dataset) val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val numClasses: Int = getNumClasses(dataset) + val numClasses = getNumClasses(dataset, $(labelCol)) if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + 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 bc2fcc03768e..03315554b817 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 @@ -381,7 +381,7 @@ class GaussianMixture @Since("2.0.0") ( val spark = dataset.sparkSession import spark.implicits._ - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val numFeatures = getNumFeatures(dataset, $(featuresCol)) require(numFeatures < GaussianMixture.MAX_NUM_FEATURES, s"GaussianMixture cannot handle more " + s"than ${GaussianMixture.MAX_NUM_FEATURES} features because the size of the covariance" + s" matrix is quadratic in the number of features.") 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 93b66f3ab700..1a97eb291005 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 @@ -18,11 +18,10 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.ml.functions.checkNonNegativeWeight import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, MetadataUtils, SchemaUtils} +import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ @@ -129,8 +128,8 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va dataset.select( col($(rawPredictionCol)), col($(labelCol)).cast(DoubleType), - if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) - else checkNonNegativeWeight(col($(weightCol)).cast(DoubleType))).rdd.map { + DatasetUtils.checkNonNegativeWeights(get(weightCol)) + ).rdd.map { case Row(rawPrediction: Vector, label: Double, weight: Double) => (rawPrediction(1), label, weight) case Row(rawPrediction: Double, label: Double, weight: Double) => diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala index fa2c25a5912a..143e26f2f749 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala @@ -18,13 +18,11 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.ml.functions.checkNonNegativeWeight import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol, HasWeightCol} import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType /** * Evaluator for clustering results. @@ -130,18 +128,13 @@ class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val uid: Str SchemaUtils.checkNumericType(schema, $(weightCol)) } - val weightColName = if (!isDefined(weightCol)) "weightCol" else $(weightCol) - - val vectorCol = DatasetUtils.columnToVector(dataset, $(featuresCol)) - val df = if (!isDefined(weightCol) || $(weightCol).isEmpty) { - dataset.select(col($(predictionCol)), - vectorCol.as($(featuresCol), dataset.schema($(featuresCol)).metadata), - lit(1.0).as(weightColName)) - } else { - dataset.select(col($(predictionCol)), - vectorCol.as($(featuresCol), dataset.schema($(featuresCol)).metadata), - checkNonNegativeWeight(col(weightColName).cast(DoubleType))) - } + val df = dataset.select( + col($(predictionCol)), + DatasetUtils.columnToVector(dataset, $(featuresCol)) + .as($(featuresCol), dataset.schema($(featuresCol)).metadata), + DatasetUtils.checkNonNegativeWeights(get(weightCol)) + .as(if (!isDefined(weightCol)) "weightCol" else $(weightCol)) + ) val metrics = new ClusteringMetrics(df) metrics.setDistanceMeasure($(distanceMeasure)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala index ffeb94927777..0106c8722971 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors} -import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.ml.util.DatasetUtils._ import org.apache.spark.sql.{Column, DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType @@ -293,7 +293,7 @@ private[evaluation] object SquaredEuclideanSilhouette extends Silhouette { predictionCol: String, featuresCol: String, weightCol: String): Map[Double, ClusterStats] = { - val numFeatures = MetadataUtils.getNumFeatures(df, featuresCol) + val numFeatures = getNumFeatures(df, featuresCol) val clustersStatsRDD = df.select( col(predictionCol).cast(DoubleType), col(featuresCol), col("squaredNorm"), col(weightCol)) .rdd @@ -509,7 +509,7 @@ private[evaluation] object CosineSilhouette extends Silhouette { featuresCol: String, predictionCol: String, weightCol: String): Map[Double, (Vector, Double)] = { - val numFeatures = MetadataUtils.getNumFeatures(df, featuresCol) + val numFeatures = getNumFeatures(df, featuresCol) val clustersStatsRDD = df.select( col(predictionCol).cast(DoubleType), col(normalizedFeaturesColName), col(weightCol)) .rdd 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 beeefde8c5fa..023987d09baa 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 @@ -18,7 +18,6 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.ml.functions.checkNonNegativeWeight import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -180,18 +179,13 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid SchemaUtils.checkColumnType(schema, $(predictionCol), DoubleType) SchemaUtils.checkNumericType(schema, $(labelCol)) - val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { - checkNonNegativeWeight(col($(weightCol)).cast(DoubleType)) - } else { - lit(1.0) - } - if ($(metricName) == "logLoss") { // probabilityCol is only needed to compute logloss require(schema.fieldNames.contains($(probabilityCol)), "probabilityCol is needed to compute logloss") } + val w = DatasetUtils.checkNonNegativeWeights(get(weightCol)) val rdd = if (schema.fieldNames.contains($(probabilityCol))) { val p = DatasetUtils.columnToVector(dataset, $(probabilityCol)) dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), w, p) 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 902869cc681b..9503e9ea11be 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 @@ -18,10 +18,9 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.ml.functions.checkNonNegativeWeight import org.apache.spark.ml.param.{BooleanParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol, HasWeightCol} -import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} +import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.RegressionMetrics import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ @@ -120,12 +119,13 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui SchemaUtils.checkNumericType(schema, $(labelCol)) val predictionAndLabelsWithWeights = dataset - .select(col($(predictionCol)).cast(DoubleType), col($(labelCol)).cast(DoubleType), - if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) - else checkNonNegativeWeight(col($(weightCol)).cast(DoubleType))) - .rdd - .map { case Row(prediction: Double, label: Double, weight: Double) => - (prediction, label, weight) } + .select( + col($(predictionCol)).cast(DoubleType), + col($(labelCol)).cast(DoubleType), + DatasetUtils.checkNonNegativeWeights(get(weightCol)) + ).rdd.map { case Row(prediction: Double, label: Double, weight: Double) => + (prediction, label, weight) + } new RegressionMetrics(predictionAndLabelsWithWeights, $(throughOrigin)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala index 7963fc88697e..5254762d210b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -346,7 +346,7 @@ private[ml] abstract class LSH[T <: LSHModel[T]] override def fit(dataset: Dataset[_]): T = { transformSchema(dataset.schema, logging = true) - val inputDim = MetadataUtils.getNumFeatures(dataset, $(inputCol)) + val inputDim = DatasetUtils.getNumFeatures(dataset, $(inputCol)) val model = createRawLSHModel(inputDim).setParent(this) copyValues(model) } 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 e8f325ec5843..85352d6bcbda 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 @@ -145,7 +145,7 @@ class RobustScaler @Since("3.0.0") (@Since("3.0.0") override val uid: String) override def fit(dataset: Dataset[_]): RobustScalerModel = { transformSchema(dataset.schema, logging = true) - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(inputCol)) + val numFeatures = DatasetUtils.getNumFeatures(dataset, $(inputCol)) val vectors = dataset.select($(inputCol)).rdd.map { case Row(vec: Vector) => require(vec.size == numFeatures, 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 e24593a01b62..1afab326dd75 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 @@ -206,7 +206,7 @@ private[ml] abstract class Selector[T <: SelectorModel[T]] val spark = dataset.sparkSession import spark.implicits._ - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val numFeatures = DatasetUtils.getNumFeatures(dataset, $(featuresCol)) val resultDF = getSelectionTestResult(dataset.toDF) def getTopIndices(k: Int): Array[Int] = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala index 7412c42986f5..3b43404072da 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala @@ -164,7 +164,7 @@ final class UnivariateFeatureSelector @Since("3.1.1")(@Since("3.1.1") override v @Since("3.1.1") override def fit(dataset: Dataset[_]): UnivariateFeatureSelectorModel = { transformSchema(dataset.schema, logging = true) - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val numFeatures = DatasetUtils.getNumFeatures(dataset, $(featuresCol)) var threshold = Double.NaN if (isSet(selectionThreshold)) { 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 874b42138727..0e571ad508ff 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 @@ -140,7 +140,7 @@ class VectorIndexer @Since("1.4.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): VectorIndexerModel = { transformSchema(dataset.schema, logging = true) - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(inputCol)) + val numFeatures = DatasetUtils.getNumFeatures(dataset, $(inputCol)) val vectorDataset = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } val maxCats = $(maxCategories) val categoryStats: VectorIndexer.CategoryStats = vectorDataset.mapPartitions { iter => diff --git a/mllib/src/main/scala/org/apache/spark/ml/functions.scala b/mllib/src/main/scala/org/apache/spark/ml/functions.scala index 43622a4f3edf..2bd7233f3acc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/functions.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/functions.scala @@ -85,10 +85,4 @@ object functions { def array_to_vector(v: Column): Column = { arrayToVectorUdf(v) } - - private[ml] def checkNonNegativeWeight = udf { - value: Double => - require(value >= 0, s"illegal weight value: $value. weight must be >= 0.0.") - value - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala index c0178ac6c764..e6e8c2f1fa4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala @@ -413,7 +413,7 @@ class FMRegressor @Since("3.0.0") ( instr.logParams(this, factorSize, fitIntercept, fitLinear, regParam, miniBatchFraction, initStd, maxIter, stepSize, tol, solver) - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val numFeatures = getNumFeatures(dataset, $(featuresCol)) instr.logNumFeatures(numFeatures) val handlePersistence = dataset.storageLevel == StorageLevel.NONE diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 10a203e9ee6b..0c58cc2449b9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -22,13 +22,12 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{BLAS, Vector} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.DatasetUtils._ +import org.apache.spark.ml.util.DatasetUtils.extractInstances import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} @@ -166,21 +165,12 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) def setWeightCol(value: String): this.type = set(weightCol, value) override protected def train(dataset: Dataset[_]): GBTRegressionModel = instrumented { instr => - - def extractInstances(df: Dataset[_]) = { - df.select( - checkRegressionLabels($(labelCol)), - checkNonNegativeWeights(get(weightCol)), - checkNonNanVectors($(featuresCol)) - ).rdd.map { case Row(l: Double, w: Double, v: Vector) => Instance(l, w, v) } - } - val withValidation = isDefined(validationIndicatorCol) && $(validationIndicatorCol).nonEmpty val (trainDataset, validationDataset) = if (withValidation) { - (extractInstances(dataset.filter(not(col($(validationIndicatorCol))))), - extractInstances(dataset.filter(col($(validationIndicatorCol))))) + (extractInstances(this, dataset.filter(not(col($(validationIndicatorCol))))), + extractInstances(this, dataset.filter(col($(validationIndicatorCol))))) } else { - (extractInstances(dataset), null) + (extractInstances(this, dataset), null) } instr.logPipelineStage(this) @@ -349,7 +339,7 @@ class GBTRegressionModel private[ml]( */ @Since("2.4.0") def evaluateEachIteration(dataset: Dataset[_], loss: String): Array[Double] = { - val data = extractInstances(dataset) + val data = extractInstances(this, dataset) GradientBoostedTrees.evaluateEachIteration(data, trees, treeWeights, convertToOldLossType(loss), OldAlgo.Regression) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 88581d03084e..6d8507239eb2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -384,7 +384,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val instr.logParams(this, labelCol, featuresCol, weightCol, offsetCol, predictionCol, linkPredictionCol, family, solver, fitIntercept, link, maxIter, regParam, tol, aggregationDepth) - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val numFeatures = getNumFeatures(dataset, $(featuresCol)) instr.logNumFeatures(numFeatures) if (numFeatures > WeightedLeastSquares.MAX_NUM_FEATURES) { 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 a53ef8c79b42..46986249e0bb 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 @@ -338,7 +338,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String } // Extract the number of features before deciding optimization solver. - val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val numFeatures = getNumFeatures(dataset, $(featuresCol)) instr.logNumFeatures(numFeatures) val instances = dataset.select( diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala index c32e901e5cda..130790ac9096 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala @@ -17,7 +17,13 @@ package org.apache.spark.ml.util +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.classification.ClassifierParams +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.shared.HasWeightCol import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -25,7 +31,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ -private[spark] object DatasetUtils { +private[spark] object DatasetUtils extends Logging { private[ml] def checkNonNanValues(colName: String, displayed: String): Column = { val casted = col(colName).cast(DoubleType) @@ -96,6 +102,26 @@ private[spark] object DatasetUtils { } } + private[ml] def extractInstances( + p: PredictorParams, + df: Dataset[_], + numClasses: Option[Int] = None): RDD[Instance] = { + val labelCol = p match { + case c: ClassifierParams => + checkClassificationLabels(c.getLabelCol, numClasses) + case _ => // TODO: there is no RegressorParams, maybe add it in the future? + checkRegressionLabels(p.getLabelCol) + } + + val weightCol = p match { + case w: HasWeightCol => checkNonNegativeWeights(w.get(w.weightCol)) + case _ => lit(1.0) + } + + df.select(labelCol, weightCol, checkNonNanVectors(p.getFeaturesCol)) + .rdd.map { case Row(l: Double, w: Double, v: Vector) => Instance(l, w, v) } + } + /** * Cast a column in a Dataset to Vector type. * @@ -138,4 +164,58 @@ private[spark] object DatasetUtils { case Row(point: Vector) => OldVectors.fromML(point) } } + + /** + * Get the number of classes. This looks in column metadata first, and if that is missing, + * then this assumes classes are indexed 0,1,...,numClasses-1 and computes numClasses + * by finding the maximum label value. + * + * Label validation (ensuring all labels are integers >= 0) needs to be handled elsewhere, + * such as in `extractLabeledPoints()`. + * + * @param dataset Dataset which contains a column [[labelCol]] + * @param maxNumClasses Maximum number of classes allowed when inferred from data. If numClasses + * is specified in the metadata, then maxNumClasses is ignored. + * @return number of classes + * @throws IllegalArgumentException if metadata does not specify numClasses, and the + * actual numClasses exceeds maxNumClasses + */ + private[ml] def getNumClasses( + dataset: Dataset[_], + labelCol: String, + maxNumClasses: Int = 100): Int = { + MetadataUtils.getNumClasses(dataset.schema(labelCol)) match { + case Some(n: Int) => n + case None => + // Get number of classes from dataset itself. + val maxLabelRow: Array[Row] = dataset + .select(max(checkClassificationLabels(labelCol, Some(maxNumClasses)))) + .take(1) + if (maxLabelRow.isEmpty || maxLabelRow(0).get(0) == null) { + throw new SparkException("ML algorithm was given empty dataset.") + } + val maxDoubleLabel: Double = maxLabelRow.head.getDouble(0) + require((maxDoubleLabel + 1).isValidInt, s"Classifier found max label value =" + + s" $maxDoubleLabel but requires integers in range [0, ... ${Int.MaxValue})") + val numClasses = maxDoubleLabel.toInt + 1 + require(numClasses <= maxNumClasses, s"Classifier inferred $numClasses from label values" + + s" in column $labelCol, but this exceeded the max numClasses ($maxNumClasses) allowed" + + s" to be inferred from values. To avoid this error for labels with > $maxNumClasses" + + s" classes, specify numClasses explicitly in the metadata; this can be done by applying" + + s" StringIndexer to the label column.") + logInfo(this.getClass.getCanonicalName + s" inferred $numClasses classes for" + + s" labelCol=$labelCol since numClasses was not specified in the column metadata.") + numClasses + } + } + + /** + * Obtain the number of features in a vector column. + * If no metadata is available, extract it from the dataset. + */ + private[ml] def getNumFeatures(dataset: Dataset[_], vectorCol: String): Int = { + MetadataUtils.getNumFeatures(dataset.schema(vectorCol)).getOrElse { + dataset.select(columnToVector(dataset, vectorCol)).head.getAs[Vector](0).size + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala index 6db0408e8d2b..631261af249f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -20,8 +20,7 @@ package org.apache.spark.ml.util import scala.collection.immutable.HashMap import org.apache.spark.ml.attribute._ -import org.apache.spark.ml.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.Dataset +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.StructField @@ -42,17 +41,6 @@ private[spark] object MetadataUtils { } } - /** - * Obtain the number of features in a vector column. - * If no metadata is available, extract it from the dataset. - */ - def getNumFeatures(dataset: Dataset[_], vectorCol: String): Int = { - getNumFeatures(dataset.schema(vectorCol)).getOrElse { - dataset.select(DatasetUtils.columnToVector(dataset, vectorCol)) - .head.getAs[Vector](0).size - } - } - /** * Examine a schema to identify the number of features in a vector column. * Returns None if the number of features is not specified. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala index 1aea4b47cd86..57cd99ecced1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala @@ -22,9 +22,8 @@ import org.apache.spark.ml.classification.ClassifierSuite.MockClassifier import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} class ClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -35,41 +34,6 @@ class ClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { labels.map { label: Double => LabeledPoint(label, Vectors.dense(0.0)) }.toDF() } - test("extractLabeledPoints") { - val c = new MockClassifier - // Valid dataset - val df0 = getTestData(Seq(0.0, 2.0, 1.0, 5.0)) - c.extractLabeledPoints(df0, 6).count() - // Invalid datasets - val df1 = getTestData(Seq(0.0, -2.0, 1.0, 5.0)) - withClue("Classifier should fail if label is negative") { - val e: SparkException = intercept[SparkException] { - c.extractLabeledPoints(df1, 6).count() - } - assert(e.getMessage.contains("given dataset with invalid label")) - } - val df2 = getTestData(Seq(0.0, 2.1, 1.0, 5.0)) - withClue("Classifier should fail if label is not an integer") { - val e: SparkException = intercept[SparkException] { - c.extractLabeledPoints(df2, 6).count() - } - assert(e.getMessage.contains("given dataset with invalid label")) - } - // extractLabeledPoints with numClasses specified - withClue("Classifier should fail if label is >= numClasses") { - val e: SparkException = intercept[SparkException] { - c.extractLabeledPoints(df0, numClasses = 5).count() - } - assert(e.getMessage.contains("given dataset with invalid label")) - } - withClue("Classifier.extractLabeledPoints should fail if numClasses <= 0") { - val e: IllegalArgumentException = intercept[IllegalArgumentException] { - c.extractLabeledPoints(df0, numClasses = 0).count() - } - assert(e.getMessage.contains("but requires numClasses > 0")) - } - } - test("getNumClasses") { val c = new MockClassifier // Valid dataset @@ -122,10 +86,8 @@ object ClassifierSuite { override def train(dataset: Dataset[_]): MockClassificationModel = throw new UnsupportedOperationException() - // Make methods public - override def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = - super.extractLabeledPoints(dataset, numClasses) - def getNumClasses(dataset: Dataset[_]): Int = super.getNumClasses(dataset) + def getNumClasses(dataset: Dataset[_]): Int = + DatasetUtils.getNumClasses(dataset, $(labelCol)) } class MockClassificationModel(override val uid: String) diff --git a/pom.xml b/pom.xml index 1ce3b43faf84..6749dfd8422d 100644 --- a/pom.xml +++ b/pom.xml @@ -132,7 +132,7 @@ 10.14.2.0 1.12.3 - 1.7.4 + 1.7.5 9.4.46.v20220331 4.0.3 0.10.0 @@ -165,8 +165,11 @@ errors building different Hadoop versions. See: SPARK-36547, SPARK-38394. --> - - 4.6.2 + + 4.6.1 true 1.9.13 diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 927384d4f1ed..01fc5d65c036 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -40,7 +40,21 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.recommendation.ALSModel.checkedCast"), // [SPARK-39110] Show metrics properties in HistoryServer environment tab - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationEnvironmentInfo.this") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationEnvironmentInfo.this"), + + // [SPARK-38775][ML] Cleanup validation functions + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.PredictionModel.extractInstances"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.Predictor.extractInstances"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.Predictor.extractLabeledPoints"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.ClassificationModel.extractInstances"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.Classifier.extractInstances"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.Classifier.extractLabeledPoints"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.Classifier.validateNumClasses"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.Classifier.validateLabel"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.Classifier.getNumClasses"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.Classifier.getNumClasses$default$2"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.OneVsRest.extractInstances"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.OneVsRestModel.extractInstances") ) // Exclude rules for 3.3.x from 3.2.0 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 568cb57976b9..6a33aae6b411 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1165,6 +1165,13 @@ object TestSettings { (Test / javaOptions) += "-Dsun.io.serialization.extendedDebugInfo=false", (Test / javaOptions) += "-Dderby.system.durability=test", (Test / javaOptions) += "-Dio.netty.tryReflectionSetAccessible=true", + (Test / javaOptions) ++= { + if ("true".equals(System.getProperty("java.net.preferIPv6Addresses"))) { + Seq("-Djava.net.preferIPv6Addresses=true") + } else { + Seq.empty + } + }, (Test / javaOptions) ++= System.getProperties.asScala.filter(_._1.startsWith("spark")) .map { case (k,v) => s"-D$k=$v" }.toSeq, (Test / javaOptions) += "-ea", diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 6676bf911937..81b6481f70ea 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -25,7 +25,7 @@ import time import gc from errno import EINTR, EAGAIN -from socket import AF_INET, SOCK_STREAM, SOMAXCONN +from socket import AF_INET, AF_INET6, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT from pyspark.worker import main as worker_main @@ -86,11 +86,17 @@ def manager(): # Create a new process group to corral our children os.setpgid(0, 0) - # Create a listening socket on the AF_INET loopback interface - listen_sock = socket.socket(AF_INET, SOCK_STREAM) - listen_sock.bind(("127.0.0.1", 0)) - listen_sock.listen(max(1024, SOMAXCONN)) - listen_host, listen_port = listen_sock.getsockname() + # Create a listening socket on the loopback interface + if os.environ.get("SPARK_PREFER_IPV6", "false").lower() == "true": + listen_sock = socket.socket(AF_INET6, SOCK_STREAM) + listen_sock.bind(("::1", 0, 0, 0)) + listen_sock.listen(max(1024, SOMAXCONN)) + listen_host, listen_port, _, _ = listen_sock.getsockname() + else: + listen_sock = socket.socket(AF_INET, SOCK_STREAM) + listen_sock.bind(("127.0.0.1", 0)) + listen_sock.listen(max(1024, SOMAXCONN)) + listen_host, listen_port = listen_sock.getsockname() # re-open stdin/stdout in 'wb' mode stdin_bin = os.fdopen(sys.stdin.fileno(), "rb", 4) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index a41ccfafde4e..aee206dd6b3e 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -193,8 +193,10 @@ def local_connect_and_auth(port, auth_secret): sock = None errors = [] # Support for both IPv4 and IPv6. - # On most of IPv6-ready systems, IPv6 will take precedence. - for res in socket.getaddrinfo("127.0.0.1", port, socket.AF_UNSPEC, socket.SOCK_STREAM): + addr = "127.0.0.1" + if os.environ.get("SPARK_PREFER_IPV6", "false").lower() == "true": + addr = "::1" + for res in socket.getaddrinfo(addr, port, socket.AF_UNSPEC, socket.SOCK_STREAM): af, socktype, proto, _, sa = res try: sock = socket.socket(af, socktype, proto) diff --git a/python/pyspark/pandas/supported_api_gen.py b/python/pyspark/pandas/supported_api_gen.py index 392b54080202..019ed0ce2542 100644 --- a/python/pyspark/pandas/supported_api_gen.py +++ b/python/pyspark/pandas/supported_api_gen.py @@ -135,11 +135,23 @@ def _create_supported_by_module( # module not implemented return {} - pd_funcs = dict([m for m in getmembers(pd_module, isfunction) if not m[0].startswith("_")]) + pd_funcs = dict( + [ + m + for m in getmembers(pd_module, isfunction) + if not m[0].startswith("_") and m[0] in pd_module.__dict__ + ] + ) if not pd_funcs: return {} - ps_funcs = dict([m for m in getmembers(ps_module, isfunction) if not m[0].startswith("_")]) + ps_funcs = dict( + [ + m + for m in getmembers(ps_module, isfunction) + if not m[0].startswith("_") and m[0] in ps_module.__dict__ + ] + ) return _organize_by_implementation_status( module_name, pd_funcs, ps_funcs, pd_module_group, ps_module_group diff --git a/python/pyspark/pandas/tests/test_indexops_spark.py b/python/pyspark/pandas/tests/test_indexops_spark.py index 88bf395b0477..275ef77f71b3 100644 --- a/python/pyspark/pandas/tests/test_indexops_spark.py +++ b/python/pyspark/pandas/tests/test_indexops_spark.py @@ -39,7 +39,7 @@ def test_series_transform_negative(self): ): self.psser.spark.transform(lambda scol: 1) - with self.assertRaisesRegex(AnalysisException, "Column.*non-existent.*does not exist"): + with self.assertRaisesRegex(AnalysisException, ".*UNRESOLVED_COLUMN.*`non-existent`.*"): self.psser.spark.transform(lambda scol: F.col("non-existent")) def test_multiindex_transform_negative(self): @@ -59,7 +59,7 @@ def test_series_apply_negative(self): ): self.psser.spark.apply(lambda scol: 1) - with self.assertRaisesRegex(AnalysisException, "Column.*non-existent.*does not exist"): + with self.assertRaisesRegex(AnalysisException, ".*UNRESOLVED_COLUMN.*`non-existent`.*"): self.psser.spark.transform(lambda scol: F.col("non-existent")) diff --git a/python/pyspark/sql/tests/test_utils.py b/python/pyspark/sql/tests/test_utils.py index f69fefc24ebd..6f504583af74 100644 --- a/python/pyspark/sql/tests/test_utils.py +++ b/python/pyspark/sql/tests/test_utils.py @@ -36,7 +36,7 @@ def test_capture_user_friendly_exception(self): try: self.spark.sql("select `中文字段`") except AnalysisException as e: - self.assertRegex(str(e), "Column '`中文字段`' does not exist") + self.assertRegex(str(e), ".*UNRESOLVED_COLUMN.*`中文字段`.*") def test_spark_upgrade_exception(self): # SPARK-32161 : Test case to Handle SparkUpgradeException in pythonic way @@ -72,7 +72,7 @@ def test_get_error_class_state(self): try: self.spark.sql("""SELECT a""") except AnalysisException as e: - self.assertEquals(e.getErrorClass(), "MISSING_COLUMN") + self.assertEquals(e.getErrorClass(), "UNRESOLVED_COLUMN") self.assertEquals(e.getSqlState(), "42000") 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 343f20fc94db..6158fbf8cd1a 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 @@ -451,7 +451,11 @@ private[yarn] class YarnAllocator( // resources on those nodes for earlier allocateResource calls, so notifying driver // to put those executors in decommissioning state allocateResponse.getUpdatedNodes.asScala.filter (node => - node.getNodeState == NodeState.DECOMMISSIONING && + // SPARK-39491: Hadoop 2.7 does not support `NodeState.DECOMMISSIONING`, + // there change to use string comparison instead for compilation. + // Should revert to `node.getNodeState == NodeState.DECOMMISSIONING` when + // Hadoop 2.7 is no longer supported. + node.getNodeState.toString.equals("DECOMMISSIONING") && !decommissioningNodesCache.containsKey(getHostAddress(node))) .foreach { node => val host = getHostAddress(node) 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 c33a10f2e1cd..2c7d7ac26c53 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 @@ -48,7 +48,7 @@ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.DecommissionExecutorsOnHost -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, VersionUtils} class MockResolver extends SparkRackResolver(SparkHadoopUtil.get.conf) { @@ -746,6 +746,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter } test("Test YARN container decommissioning") { + assume(VersionUtils.isHadoop3) val rmClient: AMRMClient[ContainerRequest] = AMRMClient.createAMRMClient() val rmClientSpy = spy(rmClient) val allocateResponse = mock(classOf[AllocateResponse]) @@ -794,7 +795,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter // host1 is now in DECOMMISSIONING state val httpAddress1 = "host1:420" - when(nodeReport.getNodeState).thenReturn(NodeState.DECOMMISSIONING) + when(nodeReport.getNodeState).thenReturn(NodeState.valueOf("DECOMMISSIONING")) when(nodeReport.getNodeId).thenReturn(nodeId) when(nodeId.getHost).thenReturn("host1") when(allocateResponse.getUpdatedNodes).thenReturn(nodeReportList) 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 9fe9d490539c..931a0fcf77f0 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 @@ -3419,10 +3419,9 @@ class Analyzer(override val catalogManager: CatalogManager) i.userSpecifiedCols, "in the column list", resolver) i.userSpecifiedCols.map { col => - i.table.resolve(Seq(col), resolver) - .getOrElse(i.failAnalysis( - errorClass = "MISSING_COLUMN", - messageParameters = Array(col, i.table.output.map(_.name).mkString(", ")))) + i.table.resolve(Seq(col), resolver).getOrElse( + throw QueryCompilationErrors.unresolvedColumnError( + col, i.table.output.map(_.name), i.origin)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 416e3a2b8349..f9f8b590a311 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -179,9 +179,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { val missingCol = a.sql val candidates = operator.inputSet.toSeq.map(_.qualifiedName) val orderedCandidates = StringUtils.orderStringsBySimilarity(missingCol, candidates) - a.failAnalysis( - errorClass = "MISSING_COLUMN", - messageParameters = Array(missingCol, orderedCandidates.mkString(", "))) + throw QueryCompilationErrors.unresolvedColumnError( + missingCol, orderedCandidates, a.origin) case s: Star => withPosition(s) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 1079f0a333dd..d305b4d37005 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -452,13 +452,17 @@ case class Inline(child: Expression) extends UnaryExpression with CollectionGene private lazy val numFields = elementSchema.fields.length + private lazy val generatorNullRow = new GenericInternalRow(elementSchema.length) + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { val inputArray = child.eval(input).asInstanceOf[ArrayData] if (inputArray == null) { Nil } else { - for (i <- 0 until inputArray.numElements()) - yield inputArray.getStruct(i, numFields) + for (i <- 0 until inputArray.numElements()) yield { + val s = inputArray.getStruct(i, numFields) + if (s == null) generatorNullRow else s + } } } 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 11d682940239..32045ff5a521 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 @@ -251,6 +251,8 @@ case class Except( override protected lazy val validConstraints: ExpressionSet = leftConstraints + override def maxRows: Option[Long] = left.maxRows + override protected def withNewChildrenInternal( newLeft: LogicalPlan, newRight: LogicalPlan): Except = copy(left = newLeft, right = newRight) } @@ -758,6 +760,9 @@ case class Sort( child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output override def maxRows: Option[Long] = child.maxRows + override def maxRowsPerPartition: Option[Long] = { + if (global) maxRows else child.maxRowsPerPartition + } override def outputOrdering: Seq[SortOrder] = order final override val nodePatterns: Seq[TreePattern] = Seq(SORT) override protected def withNewChildInternal(newChild: LogicalPlan): Sort = copy(child = newChild) @@ -1163,6 +1168,19 @@ case class Expand( override lazy val references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) + override def maxRows: Option[Long] = child.maxRows match { + case Some(m) => + val n = BigInt(m) * projections.length + if (n.isValidLong) Some(n.toLong) else None + case _ => None + } + override def maxRowsPerPartition: Option[Long] = child.maxRowsPerPartition match { + case Some(m) => + val n = BigInt(m) * projections.length + if (n.isValidLong) Some(n.toLong) else None + case _ => maxRows + } + override def metadataOutput: Seq[Attribute] = Nil override def producedAttributes: AttributeSet = AttributeSet(output diff child.output) @@ -1432,11 +1450,15 @@ case class Sample( s"Sampling fraction ($fraction) must be on interval [0, 1] without replacement") } + // when withReplacement is true, PoissonSampler is applied in SampleExec, + // which may output more rows than child. override def maxRows: Option[Long] = { - // when withReplacement is true, PoissonSampler is applied in SampleExec, - // which may output more rows than child.maxRows. if (withReplacement) None else child.maxRows } + override def maxRowsPerPartition: Option[Long] = { + if (withReplacement) None else child.maxRowsPerPartition + } + override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: LogicalPlan): Sample = @@ -1626,6 +1648,8 @@ case class CollectMetrics( name.nonEmpty && metrics.nonEmpty && metrics.forall(_.resolved) && childrenResolved } + override def maxRows: Option[Long] = child.maxRows + override def maxRowsPerPartition: Option[Long] = child.maxRowsPerPartition override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: LogicalPlan): CollectMetrics = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 2a8692efd0d1..4ee53c56f69e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -144,6 +144,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { s"side of the join. The $side-side columns: [${plan.output.map(_.name).mkString(", ")}]") } + def unresolvedColumnError( + colName: String, candidates: Seq[String], origin: Origin): Throwable = { + val candidateIds = candidates.map(candidate => toSQLId(candidate)) + new AnalysisException( + errorClass = "UNRESOLVED_COLUMN", + messageParameters = Array(toSQLId(colName), candidateIds.mkString(", ")), + origin = origin) + } + def dataTypeMismatchForDeserializerError( dataType: DataType, desiredType: String): Throwable = { new AnalysisException( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala index 558401cb4e93..345fc8e0232c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.errors import java.util.Locale +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, toPrettySQL} import org.apache.spark.sql.types.{DataType, DoubleType, FloatType} @@ -72,7 +73,7 @@ private[sql] trait QueryErrorsBase { } def toSQLId(parts: String): String = { - toSQLId(parts.split("\\.")) + toSQLId(UnresolvedAttribute.parseAttributeName(parts)) } def toSQLType(t: DataType): String = { 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 58cc4418ab02..4ac665f9d878 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 @@ -297,16 +297,16 @@ class AnalysisErrorSuite extends AnalysisTest { errorClassTest( "unresolved attributes", testRelation.select($"abcd"), - "MISSING_COLUMN", - Array("abcd", "a")) + "UNRESOLVED_COLUMN", + Array("`abcd`", "`a`")) errorClassTest( "unresolved attributes with a generated name", testRelation2.groupBy($"a")(max($"b")) .where(sum($"b") > 0) .orderBy($"havingCondition".asc), - "MISSING_COLUMN", - Array("havingCondition", "max(b)")) + "UNRESOLVED_COLUMN", + Array("`havingCondition`", "`max(b)`")) errorTest( "unresolved star expansion in max", @@ -321,8 +321,8 @@ class AnalysisErrorSuite extends AnalysisTest { errorClassTest( "sorting by attributes are not from grouping expressions", testRelation2.groupBy($"a", $"c")($"a", $"c", count($"a").as("a3")).orderBy($"b".asc), - "MISSING_COLUMN", - Array("b", "a, c, a3")) + "UNRESOLVED_COLUMN", + Array("`b`", "`a`, `c`, `a3`")) errorTest( "non-boolean filters", @@ -415,8 +415,8 @@ class AnalysisErrorSuite extends AnalysisTest { "SPARK-9955: correct error message for aggregate", // When parse SQL string, we will wrap aggregate expressions with UnresolvedAlias. testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))), - "MISSING_COLUMN", - Array("bad_column", "a, b, c, d, e")) + "UNRESOLVED_COLUMN", + Array("`bad_column`", "`a`, `b`, `c`, `d`, `e`")) errorTest( "slide duration greater than window in time window", @@ -836,7 +836,8 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "SPARK-34920: error code to error message", testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))), - "Column 'bad_column' does not exist. Did you mean one of the following? [a, b, c, d, e]" + "[UNRESOLVED_COLUMN] A column or function parameter with name `bad_column` cannot be " + + "resolved. Did you mean one of the following? [`a`, `b`, `c`, `d`, `e`]" :: Nil) test("SPARK-35080: Unsupported correlated equality predicates in subquery") { 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 5c3f4b5f5585..1b397935a898 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 @@ -103,8 +103,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisErrorClass( Project(Seq(UnresolvedAttribute("tBl.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), - "MISSING_COLUMN", - Array("tBl.a", "TbL.a")) + "UNRESOLVED_COLUMN", + Array("`tBl`.`a`", "`TbL`.`a`")) checkAnalysisWithoutViewWrapper( Project(Seq(UnresolvedAttribute("TbL.a")), @@ -711,8 +711,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("CTE with non-existing column alias") { assertAnalysisErrorClass(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"), - "MISSING_COLUMN", - Array("y", "t.x")) + "UNRESOLVED_COLUMN", + Array("`y`", "`t`.`x`")) } test("CTE with non-matching column alias") { @@ -1149,8 +1149,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { |GROUP BY c.x |ORDER BY c.x + c.y |""".stripMargin), - "MISSING_COLUMN", - Array("c.y", "x")) + "UNRESOLVED_COLUMN", + Array("`c`.`y`", "`x`")) } test("SPARK-38118: Func(wrong_type) in the HAVING clause should throw data mismatch error") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index e94c32f9b2fe..11dcae12406d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -133,34 +133,34 @@ class ResolveSubquerySuite extends AnalysisTest { // TODO: support accessing columns from outer outer query. assertAnalysisErrorClass( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b", $"c"))), - "MISSING_COLUMN", - Array("a", "")) + "UNRESOLVED_COLUMN", + Array("`a`", "")) } test("lateral subquery with unresolvable attributes") { // SELECT * FROM t1, LATERAL (SELECT a, c) assertAnalysisErrorClass( lateralJoin(t1, t0.select($"a", $"c")), - "MISSING_COLUMN", - Array("c", "") + "UNRESOLVED_COLUMN", + Array("`c`", "") ) // SELECT * FROM t1, LATERAL (SELECT a, b, c, d FROM t2) assertAnalysisErrorClass( lateralJoin(t1, t2.select($"a", $"b", $"c", $"d")), - "MISSING_COLUMN", - Array("d", "b, c") + "UNRESOLVED_COLUMN", + Array("`d`", "`b`, `c`") ) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.a)) assertAnalysisErrorClass( lateralJoin(t1, lateralJoin(t2, t0.select($"t1.a"))), - "MISSING_COLUMN", - Array("t1.a", "") + "UNRESOLVED_COLUMN", + Array("`t1`.`a`", "") ) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b)) assertAnalysisErrorClass( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b"))), - "MISSING_COLUMN", - Array("a", "") + "UNRESOLVED_COLUMN", + Array("`a`", "") ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala index 1df402d958ea..7fbedc7b312b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala @@ -688,7 +688,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { LessThanOrEqual(UnresolvedAttribute(Seq("a")), Literal(15.0d))) assertNotResolved(parsedPlan) - assertAnalysisErrorClass(parsedPlan, "MISSING_COLUMN", Array("a", "x, y")) + assertAnalysisErrorClass(parsedPlan, "UNRESOLVED_COLUMN", Array("`a`", "`x`, `y`")) val tableAcceptAnySchema = TestRelationAcceptAnySchema(StructType(Seq( StructField("x", DoubleType, nullable = false), @@ -697,7 +697,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { val parsedPlan2 = OverwriteByExpression.byPosition(tableAcceptAnySchema, query, LessThanOrEqual(UnresolvedAttribute(Seq("a")), Literal(15.0d))) assertNotResolved(parsedPlan2) - assertAnalysisErrorClass(parsedPlan2, "MISSING_COLUMN", Array("a", "x, y")) + assertAnalysisErrorClass(parsedPlan2, "UNRESOLVED_COLUMN", Array("`a`", "`x`, `y`")) } test("SPARK-36498: reorder inner fields with byName mode") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala index 5dac35a33a6b..1d533e9d0d41 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala @@ -113,4 +113,30 @@ class LogicalPlanSuite extends SparkFunSuite { assert(query.maxRows.isEmpty) assert(query.maxRowsPerPartition.isEmpty) } + + test("SPARK-37961: add maxRows/maxRowsPerPartition for some logical nodes") { + val range = Range(0, 100, 1, 3) + assert(range.maxRows === Some(100)) + assert(range.maxRowsPerPartition === Some(34)) + + val sort = Sort(Seq('id.asc), false, range) + assert(sort.maxRows === Some(100)) + assert(sort.maxRowsPerPartition === Some(34)) + val sort2 = Sort(Seq('id.asc), true, range) + assert(sort2.maxRows === Some(100)) + assert(sort2.maxRowsPerPartition === Some(100)) + + val c1 = Literal(1).as('a).toAttribute.newInstance().withNullability(true) + val c2 = Literal(2).as('b).toAttribute.newInstance().withNullability(true) + val expand = Expand( + Seq(Seq(Literal(null), 'b), Seq('a, Literal(null))), + Seq(c1, c2), + sort.select('id as 'a, 'id + 1 as 'b)) + assert(expand.maxRows === Some(200)) + assert(expand.maxRowsPerPartition === Some(68)) + + val sample = Sample(0.1, 0.9, false, 42, expand) + assert(sample.maxRows === Some(200)) + assert(sample.maxRowsPerPartition === Some(68)) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql b/sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql index e3799de5ff71..c02bc8c6063f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql @@ -13,3 +13,7 @@ WITH tmp AS ( SELECT id FROM range(4) ) SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp); + +-- SPARK-36979 +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery; +SELECT * FROM testData, LATERAL (SELECT * FROM testData) LIMIT 1; diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out index 0dc27fdca15e..9d0b9e444681 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out @@ -158,7 +158,7 @@ SELECT db1.t1.i1 FROM t1, mydb2.t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'db1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1, spark_catalog.mydb2.t1.i1]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `db1`.`t1`.`i1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]; line 1 pos 7 -- !query @@ -183,7 +183,7 @@ SELECT mydb1.t1 FROM t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'mydb1.t1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `mydb1`.`t1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb1`.`t1`.`i1`]; line 1 pos 7 -- !query @@ -201,7 +201,7 @@ SELECT t1 FROM mydb1.t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `t1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb1`.`t1`.`i1`]; line 1 pos 7 -- !query @@ -218,7 +218,7 @@ SELECT mydb1.t1.i1 FROM t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'mydb1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `mydb1`.`t1`.`i1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb2`.`t1`.`i1`]; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 09876a057c94..b84649946eb1 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -164,7 +164,7 @@ SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'non_existing' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 21 +[UNRESOLVED_COLUMN] A column or function parameter with name `non_existing` cannot be resolved. Did you mean one of the following? [`testdata`.`a`, `testdata`.`b`]; line 1 pos 21 -- !query @@ -208,7 +208,7 @@ SELECT a AS k, COUNT(b) FROM testData GROUP BY k struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 47 +[UNRESOLVED_COLUMN] A column or function parameter with name `k` cannot be resolved. Did you mean one of the following? [`testdata`.`a`, `testdata`.`b`]; line 1 pos 47 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out index 63446155938c..a4df85b4a49b 100644 --- a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out @@ -266,7 +266,7 @@ SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't2.c1' does not exist. Did you mean one of the following? []; line 1 pos 50 +[UNRESOLVED_COLUMN] A column or function parameter with name `t2`.`c1` cannot be resolved. Did you mean one of the following? []; line 1 pos 50 -- !query @@ -384,7 +384,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't1.c1' does not exist. Did you mean one of the following? []; line 1 pos 61 +[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`c1` cannot be resolved. Did you mean one of the following? []; line 1 pos 61 -- !query @@ -393,7 +393,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'c2' does not exist. Did you mean one of the following? []; line 1 pos 70 +[UNRESOLVED_COLUMN] A column or function parameter with name `c2` cannot be resolved. Did you mean one of the following? []; line 1 pos 70 -- !query @@ -420,7 +420,7 @@ SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)) struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't1.c1' does not exist. Did you mean one of the following? [spark_catalog.default.t2.c1, spark_catalog.default.t2.c2]; line 1 pos 73 +[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`c1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`default`.`t2`.`c1`, `spark_catalog`.`default`.`t2`.`c2`]; line 1 pos 73 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out index fc840486daf2..83afea9d5db2 100644 --- a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out @@ -229,7 +229,7 @@ SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2) struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'nt2.k' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.k, __auto_generated_subquery_name.v1, __auto_generated_subquery_name.v2]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `nt2`.`k` cannot be resolved. Did you mean one of the following? [`__auto_generated_subquery_name`.`k`, `__auto_generated_subquery_name`.`v1`, `__auto_generated_subquery_name`.`v2`]; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/results/non-excludable-rule.sql.out index b45e35ea4d8d..6a683995cc33 100644 --- a/sql/core/src/test/resources/sql-tests/results/non-excludable-rule.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/non-excludable-rule.sql.out @@ -37,3 +37,19 @@ SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp) struct -- !query output 2 + + +-- !query +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery + + +-- !query +SELECT * FROM testData, LATERAL (SELECT * FROM testData) LIMIT 1 +-- !query schema +struct +-- !query output +1 1 1 1 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 613ced1dce24..f739131c6277 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 @@ -229,7 +229,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 +[UNRESOLVED_COLUMN] A column or function parameter with name `year` cannot be resolved. Did you mean one of the following? [`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`]; line 4 pos 0 -- !query @@ -323,7 +323,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15 +[UNRESOLVED_COLUMN] A column or function parameter with name `s` cannot be resolved. Did you mean one of the following? [`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`]; line 4 pos 15 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index eeffa07e11ec..a0978af36064 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -495,4 +495,4 @@ from tenk1 o struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 63 +[UNRESOLVED_COLUMN] A column or function parameter with name `o`.`unique1` cannot be resolved. Did you mean one of the following? [`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`, `i`.`stringu1`, `i`.`ten`, `i`.`odd`, `i`.`string4`, `i`.`stringu2`, `i`.`tenthous`, `i`.`twenty`, `i`.`two`, `i`.`thousand`, `i`.`fivethous`, `i`.`twothousand`]; line 2 pos 63 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index cb338bbc4cd9..f9a02dbd41e1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -63,7 +63,7 @@ CREATE VIEW key_dependent_view_no_cols AS struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'FROM' does not exist. Did you mean one of the following? []; line 2 pos 10 +[UNRESOLVED_COLUMN] A column or function parameter with name `FROM` cannot be resolved. Did you mean one of the following? []; line 2 pos 10 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out index 9228558c87a6..465f1d3b0415 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out @@ -3245,7 +3245,7 @@ select * from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 +[UNRESOLVED_COLUMN] A column or function parameter with name `y`.`f1` cannot be resolved. Did you mean one of the following? [`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`]; line 2 pos 63 -- !query @@ -3264,7 +3264,7 @@ select t1.uunique1 from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`stringu1`, `t1`.`even`, `t1`.`four`, `t1`.`string4`, `t2`.`stringu1`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`even`, `t2`.`four`, `t1`.`odd`, `t2`.`string4`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`thousand`, `t1`.`twenty`, `t1`.`two`, `t1`.`fivethous`, `t2`.`odd`, `t2`.`thousand`, `t2`.`twenty`, `t2`.`two`, `t2`.`fivethous`, `t1`.`twothousand`, `t2`.`twothousand`]; line 1 pos 7 -- !query @@ -3274,7 +3274,7 @@ select t2.uunique1 from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `t2`.`uunique1` cannot be resolved. Did you mean one of the following? [`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`, `t1`.`hundred`, `t2`.`stringu1`, `t2`.`even`, `t2`.`four`, `t2`.`string4`, `t1`.`stringu1`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`even`, `t1`.`four`, `t2`.`odd`, `t1`.`string4`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`thousand`, `t2`.`twenty`, `t2`.`two`, `t2`.`fivethous`, `t1`.`odd`, `t1`.`thousand`, `t1`.`twenty`, `t1`.`two`, `t1`.`fivethous`, `t2`.`twothousand`, `t1`.`twothousand`]; line 1 pos 7 -- !query @@ -3284,7 +3284,7 @@ select uunique1 from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`, `t2`.`even`, `t1`.`four`, `t2`.`four`, `t1`.`ten`, `t2`.`ten`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`odd`, `t2`.`odd`, `t1`.`two`, `t2`.`two`, `t1`.`stringu1`, `t2`.`stringu1`, `t1`.`twenty`, `t2`.`twenty`, `t1`.`string4`, `t2`.`string4`, `t1`.`stringu2`, `t2`.`stringu2`, `t1`.`tenthous`, `t2`.`tenthous`, `t1`.`thousand`, `t2`.`thousand`, `t1`.`fivethous`, `t2`.`fivethous`, `t1`.`twothousand`, `t2`.`twothousand`]; line 1 pos 7 -- !query @@ -3484,7 +3484,7 @@ select f1,g from int4_tbl a, (select f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 37 +[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 37 -- !query @@ -3493,7 +3493,7 @@ select f1,g from int4_tbl a, (select a.f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 37 +[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 37 -- !query @@ -3502,7 +3502,7 @@ select f1,g from int4_tbl a cross join (select f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 47 +[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 47 -- !query @@ -3511,7 +3511,7 @@ select f1,g from int4_tbl a cross join (select a.f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 47 +[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 47 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out index 11b278aa42aa..4b8945033dfe 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out @@ -149,7 +149,7 @@ SELECT 1 AS one FROM test_having HAVING a > 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 40 +[UNRESOLVED_COLUMN] A column or function parameter with name `a` cannot be resolved. Did you mean one of the following? [`one`]; line 1 pos 40 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out index 41147e2b7799..eea7c3de2033 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out @@ -119,7 +119,7 @@ SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [count(1)]; line 1 pos 61 +[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`count(1)`]; line 1 pos 61 -- !query @@ -324,7 +324,7 @@ SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [count(a)]; line 1 pos 61 +[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`count(a)`]; line 1 pos 61 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out index 24ebd227e0fb..18bc925cefc2 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out @@ -565,7 +565,7 @@ SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'q2' does not exist. Did you mean one of the following? [int8_tbl.q1]; line 1 pos 64 +[UNRESOLVED_COLUMN] A column or function parameter with name `q2` cannot be resolved. Did you mean one of the following? [`int8_tbl`.`q1`]; line 1 pos 64 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index f46bfd6c885a..e454dc5e041f 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -33,7 +33,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `(a)?+.+` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 7 -- !query @@ -42,7 +42,7 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't.`(a)?+.+`' does not exist. Did you mean one of the following? [t.A, t.B, t.c, t.d]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `t`.`(a)?+.+` cannot be resolved. Did you mean one of the following? [`t`.`A`, `t`.`B`, `t`.`c`, `t`.`d`]; line 1 pos 7 -- !query @@ -51,7 +51,7 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column '`(a|b)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `(a|b)` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 7 -- !query @@ -60,7 +60,7 @@ SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `(a|b)?+.+` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 7 -- !query @@ -69,7 +69,7 @@ SELECT SUM(`(a|b)?+.+`) FROM testData2 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 +[UNRESOLVED_COLUMN] A column or function parameter with name `(a|b)?+.+` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 11 -- !query @@ -78,7 +78,7 @@ SELECT SUM(`(a)`) FROM testData2 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column '`(a)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 +[UNRESOLVED_COLUMN] A column or function parameter with name `(a)` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 11 -- !query @@ -298,7 +298,7 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)` struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column '`(a)`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38 +[UNRESOLVED_COLUMN] A column or function parameter with name `(a)` cannot be resolved. Did you mean one of the following? [`testdata3`.`a`, `testdata3`.`b`]; line 1 pos 38 -- !query @@ -307,4 +307,4 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38 +[UNRESOLVED_COLUMN] A column or function parameter with name `(a)?+.+` cannot be resolved. Did you mean one of the following? [`testdata3`.`a`, `testdata3`.`b`]; line 1 pos 38 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index cf6b57efa2fc..b358fed2663e 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -134,4 +134,4 @@ ON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a) struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't1a' does not exist. Did you mean one of the following? [t2.t2a, t2.t2b, t2.t2c]; line 4 pos 44 +[UNRESOLVED_COLUMN] A column or function parameter with name `t1a` cannot be resolved. Did you mean one of the following? [`t2`.`t2a`, `t2`.`t2b`, `t2`.`t2c`]; line 4 pos 44 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index c72e5a309bb4..5aaddbf91c22 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -57,7 +57,7 @@ SELECT a AS col1, b AS col2 FROM testData AS t(c, d) struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [t.c, t.d]; line 1 pos 7 +[UNRESOLVED_COLUMN] A column or function parameter with name `a` cannot be resolved. Did you mean one of the following? [`t`.`c`, `t`.`d`]; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index 6d9ab399aca4..c646e9993547 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -486,4 +486,4 @@ from tenk1 o struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 67 +[UNRESOLVED_COLUMN] A column or function parameter with name `o`.`unique1` cannot be resolved. Did you mean one of the following? [`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`, `i`.`stringu1`, `i`.`ten`, `i`.`odd`, `i`.`string4`, `i`.`stringu2`, `i`.`tenthous`, `i`.`twenty`, `i`.`two`, `i`.`thousand`, `i`.`fivethous`, `i`.`twothousand`]; line 2 pos 67 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out index 870a15379712..b95cbc00f35c 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out @@ -3273,7 +3273,7 @@ select * from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72 +[UNRESOLVED_COLUMN] A column or function parameter with name `y`.`f1` cannot be resolved. Did you mean one of the following? [`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`]; line 2 pos 72 -- !query @@ -3292,7 +3292,7 @@ select udf(t1.uunique1) from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11 +[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`stringu1`, `t1`.`even`, `t1`.`four`, `t1`.`string4`, `t2`.`stringu1`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`even`, `t2`.`four`, `t1`.`odd`, `t2`.`string4`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`thousand`, `t1`.`twenty`, `t1`.`two`, `t1`.`fivethous`, `t2`.`odd`, `t2`.`thousand`, `t2`.`twenty`, `t2`.`two`, `t2`.`fivethous`, `t1`.`twothousand`, `t2`.`twothousand`]; line 1 pos 11 -- !query @@ -3302,7 +3302,7 @@ select udf(udf(t2.uunique1)) from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 15 +[UNRESOLVED_COLUMN] A column or function parameter with name `t2`.`uunique1` cannot be resolved. Did you mean one of the following? [`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`, `t1`.`hundred`, `t2`.`stringu1`, `t2`.`even`, `t2`.`four`, `t2`.`string4`, `t1`.`stringu1`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`even`, `t1`.`four`, `t2`.`odd`, `t1`.`string4`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`thousand`, `t2`.`twenty`, `t2`.`two`, `t2`.`fivethous`, `t1`.`odd`, `t1`.`thousand`, `t1`.`twenty`, `t1`.`two`, `t1`.`fivethous`, `t2`.`twothousand`, `t1`.`twothousand`]; line 1 pos 15 -- !query @@ -3312,7 +3312,7 @@ select udf(uunique1) from struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11 +[UNRESOLVED_COLUMN] A column or function parameter with name `uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`, `t2`.`even`, `t1`.`four`, `t2`.`four`, `t1`.`ten`, `t2`.`ten`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`odd`, `t2`.`odd`, `t1`.`two`, `t2`.`two`, `t1`.`stringu1`, `t2`.`stringu1`, `t1`.`twenty`, `t2`.`twenty`, `t1`.`string4`, `t2`.`string4`, `t1`.`stringu2`, `t2`.`stringu2`, `t1`.`tenthous`, `t2`.`tenthous`, `t1`.`thousand`, `t2`.`thousand`, `t1`.`fivethous`, `t2`.`fivethous`, `t1`.`twothousand`, `t2`.`twothousand`]; line 1 pos 11 -- !query @@ -3512,7 +3512,7 @@ select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 55 +[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 55 -- !query @@ -3521,7 +3521,7 @@ select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 42 +[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 42 -- !query @@ -3530,7 +3530,7 @@ select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 61 +[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 61 -- !query @@ -3539,7 +3539,7 @@ select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 60 +[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 60 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out index 3308be92072b..5e7a9b96a82d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out @@ -149,7 +149,7 @@ SELECT 1 AS one FROM test_having HAVING udf(a) > 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 44 +[UNRESOLVED_COLUMN] A column or function parameter with name `a` cannot be resolved. Did you mean one of the following? [`one`]; line 1 pos 44 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out index 305d099cc4cc..13295adab15b 100755 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out @@ -122,7 +122,7 @@ SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [udf(count(1))]; line 1 pos 75 +[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`udf(count(1))`]; line 1 pos 75 -- !query @@ -327,7 +327,7 @@ SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf( struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [udf(count(udf(a)))]; line 1 pos 80 +[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`udf(count(udf(a)))`]; line 1 pos 80 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index 753556c911b7..4ffefd1b6a86 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -199,7 +199,7 @@ SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 57 +[UNRESOLVED_COLUMN] A column or function parameter with name `k` cannot be resolved. Did you mean one of the following? [`testdata`.`a`, `testdata`.`b`]; line 1 pos 57 -- !query 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 7f92e489e22b..875e14a25df2 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 @@ -229,7 +229,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 +[UNRESOLVED_COLUMN] A column or function parameter with name `year` cannot be resolved. Did you mean one of the following? [`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`]; line 4 pos 0 -- !query @@ -323,7 +323,7 @@ PIVOT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -[MISSING_COLUMN] Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15 +[UNRESOLVED_COLUMN] A column or function parameter with name `s` cannot be resolved. Did you mean one of the following? [`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`]; line 4 pos 15 -- !query 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 4d82d110a4c5..9924fbfbf626 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 @@ -2437,8 +2437,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val ex3 = intercept[AnalysisException] { df.selectExpr("transform(a, x -> x)") } - assert(ex3.getErrorClass == "MISSING_COLUMN") - assert(ex3.messageParameters.head == "a") + assert(ex3.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex3.messageParameters.head == "`a`") } test("map_filter") { @@ -2509,8 +2509,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val ex4 = intercept[AnalysisException] { df.selectExpr("map_filter(a, (k, v) -> k > v)") } - assert(ex4.getErrorClass == "MISSING_COLUMN") - assert(ex4.messageParameters.head == "a") + assert(ex4.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex4.messageParameters.head == "`a`") } test("filter function - array for primitive type not containing null") { @@ -2669,8 +2669,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val ex4 = intercept[AnalysisException] { df.selectExpr("filter(a, x -> x)") } - assert(ex4.getErrorClass == "MISSING_COLUMN") - assert(ex4.messageParameters.head == "a") + assert(ex4.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex4.messageParameters.head == "`a`") } test("exists function - array for primitive type not containing null") { @@ -2802,8 +2802,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val ex4 = intercept[AnalysisException] { df.selectExpr("exists(a, x -> x)") } - assert(ex4.getErrorClass == "MISSING_COLUMN") - assert(ex4.messageParameters.head == "a") + assert(ex4.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex4.messageParameters.head == "`a`") } test("forall function - array for primitive type not containing null") { @@ -2949,14 +2949,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val ex4 = intercept[AnalysisException] { df.selectExpr("forall(a, x -> x)") } - assert(ex4.getErrorClass == "MISSING_COLUMN") - assert(ex4.messageParameters.head == "a") + assert(ex4.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex4.messageParameters.head == "`a`") val ex4a = intercept[AnalysisException] { df.select(forall(col("a"), x => x)) } - assert(ex4a.getErrorClass == "MISSING_COLUMN") - assert(ex4a.messageParameters.head == "a") + assert(ex4a.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex4a.messageParameters.head == "`a`") } test("aggregate function - array for primitive type not containing null") { @@ -3133,8 +3133,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val ex5 = intercept[AnalysisException] { df.selectExpr("aggregate(a, 0, (acc, x) -> x)") } - assert(ex5.getErrorClass == "MISSING_COLUMN") - assert(ex5.messageParameters.head == "a") + assert(ex5.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex5.messageParameters.head == "`a`") } test("map_zip_with function - map of primitive types") { @@ -3687,8 +3687,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val ex4 = intercept[AnalysisException] { df.selectExpr("zip_with(a1, a, (acc, x) -> x)") } - assert(ex4.getErrorClass == "MISSING_COLUMN") - assert(ex4.messageParameters.head == "a") + assert(ex4.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex4.messageParameters.head == "`a`") } private def assertValuesDoNotChangeAfterCoalesceOrUnion(v: Column): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala index 4d0dd46b9569..5052dc0fc7f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala @@ -481,7 +481,10 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { val ex = intercept[AnalysisException]( df3.join(df1, year($"df1.timeStr") === year($"df3.tsStr")) ) - assert(ex.message.contains("Column 'df1.timeStr' does not exist.")) + checkError(ex, + errorClass = "UNRESOLVED_COLUMN", + parameters = Map("objectName" -> "`df1`.`timeStr`", + "objectList" -> "`df3`.`timeStr`, `df1`.`tsStr`")) } } } 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 d7dc945ae130..4daa0a1b3b6e 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 @@ -2673,8 +2673,8 @@ class DataFrameSuite extends QueryTest val err = intercept[AnalysisException] { df.groupBy($"d", $"b").as[GroupByKey, Row] } - assert(err.getErrorClass == "MISSING_COLUMN") - assert(err.messageParameters.head == "d") + assert(err.getErrorClass == "UNRESOLVED_COLUMN") + assert(err.messageParameters.head == "`d`") } test("emptyDataFrame should be foldable") { 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 25d676f5d93b..53a510fb142f 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 @@ -404,8 +404,8 @@ class DataFrameWindowFunctionsSuite extends QueryTest val df = Seq((1, "1")).toDF("key", "value") val e = intercept[AnalysisException]( df.select($"key", count("invalid").over())) - assert(e.getErrorClass == "MISSING_COLUMN") - assert(e.messageParameters.sameElements(Array("invalid", "value, key"))) + assert(e.getErrorClass == "UNRESOLVED_COLUMN") + assert(e.messageParameters.sameElements(Array("`invalid`", "`value`, `key`"))) } test("numerical aggregate functions on string column") { 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 a9cc66f8c374..463e79166fcb 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 @@ -325,13 +325,13 @@ class DatasetSuite extends QueryTest var e = intercept[AnalysisException] { ds.select(expr("`(_1)?+.+`").as[Int]) } - assert(e.getErrorClass == "MISSING_COLUMN") + assert(e.getErrorClass == "UNRESOLVED_COLUMN") assert(e.messageParameters.head == "`(_1)?+.+`") e = intercept[AnalysisException] { ds.select(expr("`(_1|_2)`").as[Int]) } - assert(e.getErrorClass == "MISSING_COLUMN") + assert(e.getErrorClass == "UNRESOLVED_COLUMN") assert(e.messageParameters.head == "`(_1|_2)`") e = intercept[AnalysisException] { @@ -934,8 +934,8 @@ class DatasetSuite extends QueryTest val e = intercept[AnalysisException] { ds.as[ClassData2] } - assert(e.getErrorClass == "MISSING_COLUMN") - assert(e.messageParameters.sameElements(Array("c", "a, b"))) + assert(e.getErrorClass == "UNRESOLVED_COLUMN") + assert(e.messageParameters.sameElements(Array("`c`", "`a`, `b`"))) } test("runtime nullability check") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index 20d6588dd6dc..648f9cac216e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCo import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.trees.LeafLike 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.types.{IntegerType, StructType} @@ -390,7 +391,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { } } - test("SPARK-39061: inline should handle null struct") { + def testNullStruct(): Unit = { val df = sql( """select * from values |( @@ -414,6 +415,16 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { sql("select a, inline(b) from t1"), Row(1, 0, 1) :: Row(1, null, null) :: Row(1, 2, 3) :: Row(1, null, null) :: Nil) } + + test("SPARK-39061: inline should handle null struct") { + testNullStruct + } + + test("SPARK-39496: inline eval path should handle null struct") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + testNullStruct + } + } } case class EmptyGenerator() extends Generator with LeafLike[Expression] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index 11abca1ddc7c..7fd6a5dbea00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -167,8 +167,8 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { createTable("t1", cols, Seq("int", "long", "string")) val e1 = intercept[AnalysisException](sql(s"INSERT INTO t1 (c1, c2, c4) values(1, 2, 3)")) assert(e1.getMessage.contains( - "[MISSING_COLUMN] Column 'c4' does not exist. Did you mean one of the following? " + - "[c1, c2, c3]")) + "[UNRESOLVED_COLUMN] A column or function parameter with name `c4` cannot be resolved. " + + "Did you mean one of the following? [`c1`, `c2`, `c3`]")) } } 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 2bfaa5b22d5c..214306838402 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 @@ -4225,13 +4225,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(sql("""SELECT from_json(R'{"a": "\\"}', 'a string')"""), Row(Row("\\"))) } - test("SPARK-36979: Add RewriteLateralSubquery rule into nonExcludableRules") { - withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> - "org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery") { - sql("SELECT * FROM testData, LATERAL (SELECT * FROM testData)").collect() - } - } - test("TABLE SAMPLE") { withTable("test") { sql("CREATE TABLE test(c int) USING PARQUET") 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 d2357c3d6c0e..b1057fd14bc8 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 @@ -887,8 +887,8 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark withTempView("t") { Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("t") val e = intercept[AnalysisException](sql("SELECT (SELECT count(*) FROM t WHERE a = 1)")) - assert(e.getErrorClass == "MISSING_COLUMN") - assert(e.messageParameters.sameElements(Array("a", "t.i, t.j"))) + assert(e.getErrorClass == "UNRESOLVED_COLUMN") + assert(e.messageParameters.sameElements(Array("`a`", "`t`.`i`, `t`.`j`"))) } } 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 8fd7aa3e0cd1..ba8af6238938 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 @@ -725,8 +725,8 @@ class UDFSuite extends QueryTest with SharedSparkSession { .select(lit(50).as("a")) .select(struct("a").as("col")) val error = intercept[AnalysisException](df.select(myUdf(Column("col")))) - assert(error.getErrorClass == "MISSING_COLUMN") - assert(error.messageParameters.sameElements(Array("b", "a"))) + assert(error.getErrorClass == "UNRESOLVED_COLUMN") + assert(error.messageParameters.sameElements(Array("`b`", "`a`"))) } test("wrong order of input fields for case class") { 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 45c721100f7d..00bb37dcfb24 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 @@ -175,8 +175,8 @@ class DataSourceV2SQLSuite assertAnalysisErrorClass( s"DESCRIBE $t invalid_col", - "MISSING_COLUMN", - Array("invalid_col", "testcat.tbl.id, testcat.tbl.data")) + "UNRESOLVED_COLUMN", + Array("`invalid_col`", "`testcat`.`tbl`.`id`, `testcat`.`tbl`.`data`")) } } @@ -1050,8 +1050,8 @@ class DataSourceV2SQLSuite val ex = intercept[AnalysisException] { sql(s"SELECT ns1.ns2.ns3.tbl.id from $t") } - assert(ex.getErrorClass == "MISSING_COLUMN") - assert(ex.messageParameters.head == "ns1.ns2.ns3.tbl.id") + assert(ex.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex.messageParameters.head == "`ns1`.`ns2`.`ns3`.`tbl`.`id`") } } @@ -1710,18 +1710,18 @@ class DataSourceV2SQLSuite // UPDATE non-existing column assertAnalysisErrorClass( s"UPDATE $t SET dummy='abc'", - "MISSING_COLUMN", + "UNRESOLVED_COLUMN", Array( - "dummy", - "testcat.ns1.ns2.tbl.p, testcat.ns1.ns2.tbl.id, " + - "testcat.ns1.ns2.tbl.age, testcat.ns1.ns2.tbl.name")) + "`dummy`", + "`testcat`.`ns1`.`ns2`.`tbl`.`p`, `testcat`.`ns1`.`ns2`.`tbl`.`id`, " + + "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`")) assertAnalysisErrorClass( s"UPDATE $t SET name='abc' WHERE dummy=1", - "MISSING_COLUMN", + "UNRESOLVED_COLUMN", Array( - "dummy", - "testcat.ns1.ns2.tbl.p, testcat.ns1.ns2.tbl.id, " + - "testcat.ns1.ns2.tbl.age, testcat.ns1.ns2.tbl.name")) + "`dummy`", + "`testcat`.`ns1`.`ns2`.`tbl`.`p`, `testcat`.`ns1`.`ns2`.`tbl`.`id`, " + + "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`")) // UPDATE is not implemented yet. val e = intercept[UnsupportedOperationException] { @@ -2773,7 +2773,7 @@ class DataSourceV2SQLSuite assert(e2.getMessage.contains(errMsg)) } checkSubqueryError("SELECT 1 FROM non_exist", "Table or view not found: non_exist") - checkSubqueryError("SELECT col", "MISSING_COLUMN") + checkSubqueryError("SELECT col", "UNRESOLVED_COLUMN") checkSubqueryError("SELECT 1, 2", "Scalar subquery must return only one column") checkSubqueryError("SELECT * FROM VALUES (1), (2)", "MULTI_VALUE_SUBQUERY_ERROR") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 4f631292436c..06e6bec3fd15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -401,7 +401,7 @@ class QueryCompilationErrorsSuite ) } - test("MISSING_COLUMN: SELECT distinct does not work correctly " + + test("UNRESOLVED_COLUMN: SELECT distinct does not work correctly " + "if order by missing attribute") { checkAnswer( sql( @@ -424,9 +424,9 @@ class QueryCompilationErrorsSuite |order by struct.a, struct.b |""".stripMargin) }, - errorClass = "MISSING_COLUMN", - msg = """Column 'struct.a' does not exist. """ + - """Did you mean one of the following\? \[a, b\]; line 6 pos 9; + errorClass = "UNRESOLVED_COLUMN", + msg = """A column or function parameter with name `struct`.`a` cannot be resolved. """ + + """Did you mean one of the following\? \[`a`, `b`\]; line 6 pos 9; |'Sort \['struct.a ASC NULLS FIRST, 'struct.b ASC NULLS FIRST\], true |\+\- Distinct | \+\- Project \[struct#\w+\.a AS a#\w+, struct#\w+\.b AS b#\w+\] @@ -440,16 +440,17 @@ class QueryCompilationErrorsSuite matchMsg = true) } - test("MISSING_COLUMN - SPARK-21335: support un-aliased subquery") { + test("UNRESOLVED_COLUMN - SPARK-21335: support un-aliased subquery") { withTempView("v") { Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("v") checkAnswer(sql("SELECT i from (SELECT i FROM v)"), Row(1)) checkErrorClass( exception = intercept[AnalysisException](sql("SELECT v.i from (SELECT i FROM v)")), - errorClass = "MISSING_COLUMN", - msg = """Column 'v.i' does not exist. Did you mean one of the following\? """ + - """\[__auto_generated_subquery_name.i\]; line 1 pos 7; + errorClass = "UNRESOLVED_COLUMN", + msg = "A column or function parameter with name `v`.`i` cannot be resolved. " + + """Did you mean one of the following\? """ + + """\[`__auto_generated_subquery_name`.`i`\]; line 1 pos 7; |'Project \['v.i\] |\+\- SubqueryAlias __auto_generated_subquery_name | \+\- Project \[i#\w+\] 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 c1abe8e90bea..823c7e01195b 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 @@ -871,8 +871,11 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("SELECT * FROM v1") } - assert(e.getErrorClass == "MISSING_COLUMN") - assert(e.messageParameters.sameElements(Array("C1", "spark_catalog.default.t.c1"))) + checkError(e, + errorClass = "UNRESOLVED_COLUMN", + parameters = Map( + "objectName" -> "`C1`", + "objectList" -> "`spark_catalog`.`default`.`t`.`c1`")) } withSQLConf(ORDER_BY_ORDINAL.key -> "false") { checkAnswer(sql("SELECT * FROM v2"), Seq(Row(3), Row(2), Row(1))) @@ -890,8 +893,11 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("SELECT * FROM v4") } - assert(e.getErrorClass == "MISSING_COLUMN") - assert(e.messageParameters.sameElements(Array("a", "spark_catalog.default.t.c1"))) + checkError(e, + errorClass = "UNRESOLVED_COLUMN", + parameters = Map( + "objectName" -> "`a`", + "objectList" -> "`spark_catalog`.`default`.`t`.`c1`")) } withSQLConf(ANSI_ENABLED.key -> "true") { val e = intercept[ArithmeticException] { 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 17e23b9f7b1f..62dccaad1ddd 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 @@ -2619,8 +2619,8 @@ abstract class CSVSuite val ex = intercept[AnalysisException] { readback.filter($"AAA" === 2 && $"bbb" === 3).collect() } - assert(ex.getErrorClass == "MISSING_COLUMN") - assert(ex.messageParameters.head == "AAA") + assert(ex.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex.messageParameters.head == "`AAA`") } } } 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 bc7c6e56ece5..3fe9c58c957c 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 @@ -3044,8 +3044,8 @@ abstract class JsonSuite val ex = intercept[AnalysisException] { readback.filter($"AAA" === 0 && $"bbb" === 1).collect() } - assert(ex.getErrorClass == "MISSING_COLUMN") - assert(ex.messageParameters.head == "AAA") + assert(ex.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex.messageParameters.head == "`AAA`") // Schema inferring val readback2 = spark.read.json(path.getCanonicalPath) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index a1d00361dfc5..7370cf4f28b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -1024,7 +1024,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-38336 INSERT INTO statements with tables with default columns: negative tests") { object Errors { val COMMON_SUBSTRING = " has a DEFAULT value" - val COLUMN_DEFAULT_NOT_FOUND = "Column 'default' does not exist" + val COLUMN_DEFAULT_NOT_FOUND = "`default` cannot be resolved." val BAD_SUBQUERY = "cannot evaluate expression scalarsubquery() in inline table definition" } // The default value fails to analyze. @@ -1278,7 +1278,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { assert(intercept[AnalysisException] { sql("insert into t (I) select true from (select 1)") }.getMessage.contains( - "[MISSING_COLUMN] Column 'I' does not exist. Did you mean one of the following? [i, s]")) + "[UNRESOLVED_COLUMN] A column or function parameter with name `I` cannot be resolved. " + + "Did you mean one of the following? [`i`, `s`]")) } } } @@ -1757,8 +1758,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { |) """.stripMargin) } - assert(ex.getErrorClass == "MISSING_COLUMN") - assert(ex.messageParameters.head == "c3") + assert(ex.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex.messageParameters.head == "`c3`") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index 76a66cfdeb7d..e5ecc2c889c2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -123,8 +123,8 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton |) """.stripMargin) } - assert(ex.getErrorClass == "MISSING_COLUMN") - assert(ex.messageParameters.head == "c3") + assert(ex.getErrorClass == "UNRESOLVED_COLUMN") + assert(ex.messageParameters.head == "`c3`") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala index d48d70f7d4e7..6648c04a4c50 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala @@ -22,7 +22,10 @@ import scala.collection.immutable.IndexedSeq import org.apache.commons.lang3.{JavaVersion, SystemUtils} private[client] trait HiveClientVersions { - protected val versions = if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + private val testVersions = sys.env.get("SPARK_TEST_HIVE_CLIENT_VERSIONS") + protected val versions = if (testVersions.nonEmpty) { + testVersions.get.split(",").map(_.trim).filter(_.nonEmpty).toIndexedSeq + } else if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { IndexedSeq("2.0", "2.1", "2.2", "2.3", "3.0", "3.1") } else { IndexedSeq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1", "2.2", "2.3", "3.0",