From fb41b90c8914d8045691b9b752d0f86704538f42 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 2 Mar 2021 16:58:29 -0800 Subject: [PATCH 001/169] [SPARK-34361][K8S] In case of downscaling avoid killing of executors already known by the scheduler backend in the pod allocator MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR modifies the POD allocator to use the scheduler backend to get the known executors and remove those from the pending and newly created list. This is different from the normal `ExecutorAllocationManager` requested killing of executors where the `spark.dynamicAllocation.executorIdleTimeout` is used. In this case POD allocator kills the executors which should be only responsible for terminating not satisfied POD allocations (new requests where no POD state is received yet and PODs in pending state). ### Why are the changes needed? Because there is race between executor POD allocator and cluster scheduler backend. Running several experiment during downscaling we experienced a lot of killed fresh executors wich has already running task on them. The pattern in the log was the following (see executor 312 and TID 2079): ``` 21/02/01 15:12:03 INFO ExecutorMonitor: New executor 312 has registered (new total is 138) ... 21/02/01 15:12:03 INFO TaskSetManager: Starting task 247.0 in stage 4.0 (TID 2079, 100.100.18.138, executor 312, partition 247, PROCESS_LOCAL, 8777 bytes) 21/02/01 15:12:03 INFO ExecutorPodsAllocator: Deleting 3 excess pod requests (408,312,307). ... 21/02/01 15:12:04 ERROR TaskSchedulerImpl: Lost executor 312 on 100.100.18.138: The executor with id 312 was deleted by a user or the framework. 21/02/01 15:12:04 INFO TaskSetManager: Task 2079 failed because while it was being computed, its executor exited for a reason unrelated to the task. Not counting this failure towards the maximum number of failures for the task. ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? #### Manually With this change there was no executor lost with running task on it. ##### With unit test A new test is added and existing test is modified to check these cases. Closes #31513 from attilapiros/SPARK-34361. Authored-by: “attilapiros” Signed-off-by: Holden Karau (cherry picked from commit 6c5322de6176726955b4bc941f92ecaa54a7f539) Signed-off-by: Holden Karau --- .../org/apache/spark/deploy/k8s/Config.scala | 3 +- .../cluster/k8s/ExecutorPodsAllocator.scala | 66 +++++--- .../KubernetesClusterSchedulerBackend.scala | 2 +- .../k8s/ExecutorPodsAllocatorSuite.scala | 155 +++++++++++++++++- ...bernetesClusterSchedulerBackendSuite.scala | 2 +- 5 files changed, 204 insertions(+), 24 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 2eeffd840b7db..fe92fae36c36c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -237,7 +237,8 @@ private[spark] object Config extends Logging { val KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT = ConfigBuilder("spark.kubernetes.allocation.executor.timeout") - .doc("Time to wait before considering a pending executor timedout.") + .doc("Time to wait before a newly created executor POD request, which does not reached " + + "the POD pending state yet, considered timedout and will be deleted.") .version("3.1.0") .timeConf(TimeUnit.MILLISECONDS) .checkValue(value => value > 0, "Allocation executor timeout must be a positive time value.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index eb35de8759593..5fc81a6d84273 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s import java.time.Instant import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -82,9 +82,14 @@ private[spark] class ExecutorPodsAllocator( // snapshot yet. Mapped to the (ResourceProfile id, timestamp) when they were created. private val newlyCreatedExecutors = mutable.LinkedHashMap.empty[Long, (Int, Long)] + // Executor IDs that have been requested from Kubernetes but have not been detected in any POD + // snapshot yet but already known by the scheduler backend. Mapped to the ResourceProfile id. + private val schedulerKnownNewlyCreatedExecs = mutable.LinkedHashMap.empty[Long, Int] + private val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(conf) - private val hasPendingPods = new AtomicBoolean() + // visible for tests + private[k8s] val numOutstandingPods = new AtomicInteger() private var lastSnapshot = ExecutorPodsSnapshot() @@ -93,9 +98,9 @@ private[spark] class ExecutorPodsAllocator( // if they happen to come up before the deletion takes effect. @volatile private var deletedExecutorIds = Set.empty[Long] - def start(applicationId: String): Unit = { + def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { snapshotsStore.addSubscriber(podAllocationDelay) { - onNewSnapshots(applicationId, _) + onNewSnapshots(applicationId, schedulerBackend, _) } } @@ -105,7 +110,7 @@ private[spark] class ExecutorPodsAllocator( totalExpectedExecutorsPerResourceProfileId.put(rp.id, numExecs) } logDebug(s"Set total expected execs to $totalExpectedExecutorsPerResourceProfileId") - if (!hasPendingPods.get()) { + if (numOutstandingPods.get() == 0) { snapshotsStore.notifySubscribers() } } @@ -114,8 +119,19 @@ private[spark] class ExecutorPodsAllocator( private def onNewSnapshots( applicationId: String, + schedulerBackend: KubernetesClusterSchedulerBackend, snapshots: Seq[ExecutorPodsSnapshot]): Unit = { - newlyCreatedExecutors --= snapshots.flatMap(_.executorPods.keys) + val k8sKnownExecIds = snapshots.flatMap(_.executorPods.keys) + newlyCreatedExecutors --= k8sKnownExecIds + schedulerKnownNewlyCreatedExecs --= k8sKnownExecIds + + // transfer the scheduler backend known executor requests from the newlyCreatedExecutors + // to the schedulerKnownNewlyCreatedExecs + val schedulerKnownExecs = schedulerBackend.getExecutorIds().map(_.toLong).toSet + schedulerKnownNewlyCreatedExecs ++= + newlyCreatedExecutors.filterKeys(schedulerKnownExecs.contains(_)).mapValues(_._1) + newlyCreatedExecutors --= schedulerKnownNewlyCreatedExecs.keySet + // For all executors we've created against the API but have not seen in a snapshot // yet - check the current time. If the current time has exceeded some threshold, // assume that the pod was either never created (the API server never properly @@ -164,15 +180,16 @@ private[spark] class ExecutorPodsAllocator( _deletedExecutorIds = _deletedExecutorIds.filter(existingExecs.contains) } + val notDeletedPods = lastSnapshot.executorPods.filterKeys(!_deletedExecutorIds.contains(_)) // Map the pods into per ResourceProfile id so we can check per ResourceProfile, // add a fast path if not using other ResourceProfiles. val rpIdToExecsAndPodState = mutable.HashMap[Int, mutable.HashMap[Long, ExecutorPodState]]() if (totalExpectedExecutorsPerResourceProfileId.size <= 1) { rpIdToExecsAndPodState(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = - mutable.HashMap.empty ++= lastSnapshot.executorPods + mutable.HashMap.empty ++= notDeletedPods } else { - lastSnapshot.executorPods.foreach { case (execId, execPodState) => + notDeletedPods.foreach { case (execId, execPodState) => val rpId = execPodState.pod.getMetadata.getLabels.get(SPARK_RESOURCE_PROFILE_ID_LABEL).toInt val execPods = rpIdToExecsAndPodState.getOrElseUpdate(rpId, mutable.HashMap[Long, ExecutorPodState]()) @@ -190,24 +207,33 @@ private[spark] class ExecutorPodsAllocator( case _ => false } - val currentPendingExecutors = podsForRpId.filter { + val (schedulerKnownPendingExecsForRpId, currentPendingExecutorsForRpId) = podsForRpId.filter { case (_, PodPending(_)) => true case _ => false + }.partition { case (k, _) => + schedulerKnownExecs.contains(k) } // This variable is used later to print some debug logs. It's updated when cleaning up - // excess pod requests, since currentPendingExecutors is immutable. - var knownPendingCount = currentPendingExecutors.size + // excess pod requests, since currentPendingExecutorsForRpId is immutable. + var knownPendingCount = currentPendingExecutorsForRpId.size val newlyCreatedExecutorsForRpId = newlyCreatedExecutors.filter { case (_, (waitingRpId, _)) => rpId == waitingRpId } + val schedulerKnownNewlyCreatedExecsForRpId = + schedulerKnownNewlyCreatedExecs.filter { case (_, waitingRpId) => + rpId == waitingRpId + } + if (podsForRpId.nonEmpty) { logDebug(s"ResourceProfile Id: $rpId " + s"pod allocation status: $currentRunningCount running, " + - s"${currentPendingExecutors.size} pending. " + - s"${newlyCreatedExecutorsForRpId.size} unacknowledged.") + s"${currentPendingExecutorsForRpId.size} unknown pending, " + + s"${schedulerKnownPendingExecsForRpId.size} scheduler backend known pending, " + + s"${newlyCreatedExecutorsForRpId.size} unknown newly created, " + + s"${schedulerKnownNewlyCreatedExecsForRpId.size} scheduler backend known newly created.") } // It's possible that we have outstanding pods that are outdated when dynamic allocation @@ -218,8 +244,9 @@ private[spark] class ExecutorPodsAllocator( // // TODO: with dynamic allocation off, handle edge cases if we end up with more running // executors than expected. - val knownPodCount = currentRunningCount + currentPendingExecutors.size + - newlyCreatedExecutorsForRpId.size + val knownPodCount = currentRunningCount + + currentPendingExecutorsForRpId.size + schedulerKnownPendingExecsForRpId.size + + newlyCreatedExecutorsForRpId.size + schedulerKnownNewlyCreatedExecsForRpId.size if (knownPodCount > targetNum) { val excess = knownPodCount - targetNum @@ -227,7 +254,7 @@ private[spark] class ExecutorPodsAllocator( .filter { case (_, (_, createTime)) => currentTime - createTime > executorIdleTimeout }.keys.take(excess).toList - val knownPendingToDelete = currentPendingExecutors + val knownPendingToDelete = currentPendingExecutorsForRpId .filter(x => isExecutorIdleTimedOut(x._2, currentTime)) .take(excess - newlyCreatedToDelete.size) .map { case (id, _) => id } @@ -245,7 +272,7 @@ private[spark] class ExecutorPodsAllocator( .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabelIn(SPARK_EXECUTOR_ID_LABEL, toDelete.sorted.map(_.toString): _*) .delete() - newlyCreatedExecutors --= toDelete + newlyCreatedExecutors --= newlyCreatedToDelete knownPendingCount -= knownPendingToDelete.size } } @@ -276,8 +303,9 @@ private[spark] class ExecutorPodsAllocator( deletedExecutorIds = _deletedExecutorIds // Update the flag that helps the setTotalExpectedExecutors() callback avoid triggering this - // update method when not needed. - hasPendingPods.set(totalPendingCount + newlyCreatedExecutors.size > 0) + // update method when not needed. PODs known by the scheduler backend are not counted here as + // they considered running PODs and they should not block upscaling. + numOutstandingPods.set(totalPendingCount + newlyCreatedExecutors.size) } private def requestNewExecutors( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index c35a434f83ec1..d58e38ab9794a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -93,7 +93,7 @@ private[spark] class KubernetesClusterSchedulerBackend( val initExecs = Map(defaultProfile -> initialExecutors) podAllocator.setTotalExpectedExecutors(initExecs) lifecycleEventHandler.start(this) - podAllocator.start(applicationId()) + podAllocator.start(applicationId(), this) watchEvents.start(applicationId()) pollEvents.start(applicationId()) setUpExecutorConfigMap() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 349bbcd6f7883..55be80ae29c7e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -81,6 +81,9 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var executorBuilder: KubernetesExecutorBuilder = _ + @Mock + private var schedulerBackend: KubernetesClusterSchedulerBackend = _ + private var snapshotsStore: DeterministicExecutorPodsSnapshotsStore = _ private var podsAllocatorUnderTest: ExecutorPodsAllocator = _ @@ -96,12 +99,14 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock = new ManualClock(0L) podsAllocatorUnderTest = new ExecutorPodsAllocator( conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) - podsAllocatorUnderTest.start(TEST_SPARK_APP_ID) + when(schedulerBackend.getExecutorIds).thenReturn(Seq.empty) + podsAllocatorUnderTest.start(TEST_SPARK_APP_ID, schedulerBackend) } test("Initially request executors in batches. Do not request another batch if the" + " first has not finished.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> (podAllocationSize + 1))) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (nextId <- 1 to podAllocationSize) { verify(podOperations).create(podWithAttachedContainerForId(nextId)) } @@ -111,28 +116,34 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { test("Request executors in batches. Allow another batch to be requested if" + " all pending executors start running.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> (podAllocationSize + 1))) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 1)) snapshotsStore.updatePod(runningExecutor(podAllocationSize)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) snapshotsStore.updatePod(runningExecutor(podAllocationSize)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations, times(podAllocationSize + 1)).create(any(classOf[Pod])) } test("When a current batch reaches error states immediately, re-request" + " them on the next batch.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> podAllocationSize)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } val failedPod = failedExecutorWithoutDeletion(podAllocationSize) snapshotsStore.updatePod(failedPod) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) } @@ -148,9 +159,11 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { .withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1")) .thenReturn(labeledPods) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(1)) waitForExecutorPodsClock.setTime(podCreationTimeout + 1) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(labeledPods).delete() verify(podOperations).create(podWithAttachedContainerForId(2)) } @@ -174,17 +187,20 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Target 1 executor, make sure it's requested, even with an empty initial snapshot. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(1)) // Mark executor as running, verify that subsequent allocation cycle is a no-op. snapshotsStore.updatePod(runningExecutor(1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(1)).create(any()) verify(podOperations, never()).delete() // Request 3 more executors, make sure all are requested. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 4)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations).create(podWithAttachedContainerForId(2)) verify(podOperations).create(podWithAttachedContainerForId(3)) verify(podOperations).create(podWithAttachedContainerForId(4)) @@ -193,6 +209,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(runningExecutor(2)) snapshotsStore.updatePod(pendingExecutor(3)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) verify(podOperations, times(4)).create(any()) verify(podOperations, never()).delete() @@ -200,6 +217,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.advance(executorIdleTimeout * 2) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(4)).create(any()) verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "3", "4") verify(podOperations).delete() @@ -212,6 +230,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(deletedExecutor(4)) snapshotsStore.removeDeletedExecutors() snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) assert(!podsAllocatorUnderTest.isDeleted("3")) assert(!podsAllocatorUnderTest.isDeleted("4")) } @@ -279,6 +298,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.setTime(startTime) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 5)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) verify(podOperations).create(podWithAttachedContainerForId(1)) verify(podOperations).create(podWithAttachedContainerForId(2)) verify(podOperations).create(podWithAttachedContainerForId(3)) @@ -292,16 +312,139 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Newly created executors (both acknowledged and not) are protected by executorIdleTimeout podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 0)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) verify(podOperations, never()).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2", "3", "4", "5") verify(podOperations, never()).delete() // Newly created executors (both acknowledged and not) are cleaned up. waitForExecutorPodsClock.advance(executorIdleTimeout * 2) + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "3", "4")) snapshotsStore.notifySubscribers() - verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2", "3", "4", "5") + // SPARK-34361: even as 1, 3 and 4 are not timed out as they are considered as known PODs so + // this is why they are not counted into the outstanding PODs and /they are not removed even + // though executor 1 is still in pending state and executor 3 and 4 are new request without + // any state reported by kubernetes and all the three are already timed out + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "2", "5") verify(podOperations).delete() } + /** + * This test covers some downscaling and upscaling of dynamic allocation on kubernetes + * along with multiple resource profiles (default and rp) when some executors + * already know by the scheduler backend. + * + * Legend: + * + * N-: newly created not known by the scheduler backend + * N+: newly created known by the scheduler backend + * P- / P+ : pending (not know / known) by the scheduler backend + * D: deleted + * | default || rp | expected + * | || | outstanding + * | 1 | 2 | 3 || 4 | 5 | 6 | 7 | PODs + * ========================================================================================== + * 0) setTotalExpectedExecs with | N- | N- | N- || N- | N- | N- | N- | + * default->3, ro->4 | | | || | | | | 7 + * ------------------------------------------------------------------------------------------ + * 1) make 1 from each rp | N+ | N- | N- || N+ | N- | N- | N- | + * known by backend | | | || | | | | 5 + * ------------------------------------------------------------------------------------------- + * 2) some more backend known + pending | N+ | P+ | P- || N+ | P+ | P- | N- | 3 + * ------------------------------------------------------------------------------------------- + * 3) advance time with idle timeout | | | || | | | | + * setTotalExpectedExecs with | N+ | P+ | D || N+ | P+ | D | D | 0 + * default->1, rp->1 | | | || | | | | + * ------------------------------------------------------------------------------------------- + * 4) setTotalExpectedExecs with | N+ | P+ | D || N+ | P+ | D | D | 0 and + * default->2, rp->2 | | | || | | | | no new POD req. + * =========================================================================================== + * + * 5) setTotalExpectedExecs with default -> 3, rp -> 3 which will lead to creation of the new + * PODs: 8 and 9 + */ + test("SPARK-34361: scheduler backend known pods with multiple resource profiles at downscaling") { + when(podOperations + .withField("status.phase", "Pending")) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(podOperations) + when(podOperations + .withLabelIn(meq(SPARK_EXECUTOR_ID_LABEL), any())) + .thenReturn(podOperations) + + val startTime = Instant.now.toEpochMilli + waitForExecutorPodsClock.setTime(startTime) + + val rpb = new ResourceProfileBuilder() + val ereq = new ExecutorResourceRequests() + val treq = new TaskResourceRequests() + ereq.cores(4).memory("2g") + treq.cpus(2) + rpb.require(ereq).require(treq) + val rp = rpb.build() + + // 0) request 3 PODs for the default and 4 PODs for the other resource profile + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 3, rp -> 4)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 7) + verify(podOperations).create(podWithAttachedContainerForId(1, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(2, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(3, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(4, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(5, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(6, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(7, rp.id)) + + // 1) make 1 POD known by the scheduler backend for each resource profile + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "4")) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5, + "scheduler backend known PODs are not outstanding") + verify(podOperations, times(7)).create(any()) + + // 2) make 1 extra POD known by the scheduler backend for each resource profile + // and make some to pending + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "2", "4", "5")) + snapshotsStore.updatePod(pendingExecutor(2, defaultProfile.id)) + snapshotsStore.updatePod(pendingExecutor(3, defaultProfile.id)) + snapshotsStore.updatePod(pendingExecutor(5, rp.id)) + snapshotsStore.updatePod(pendingExecutor(6, rp.id)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) + verify(podOperations, times(7)).create(any()) + + // 3) downscale to 1 POD for default and 1 POD for the other resource profile + waitForExecutorPodsClock.advance(executorIdleTimeout * 2) + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 1)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations, times(7)).create(any()) + verify(podOperations, times(2)).delete() + assert(podsAllocatorUnderTest.isDeleted("3")) + assert(podsAllocatorUnderTest.isDeleted("6")) + assert(podsAllocatorUnderTest.isDeleted("7")) + + // 4) upscale to 2 PODs for default and 2 for the other resource profile but as there is still + // 2 PODs known by the scheduler backend there must be no new POD requested to be created + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 2, rp -> 2)) + snapshotsStore.notifySubscribers() + verify(podOperations, times(7)).create(any()) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations, times(7)).create(any()) + + // 5) requesting 1 more executor for each resource + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 3, rp -> 3)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) + verify(podOperations, times(9)).create(any()) + verify(podOperations).create(podWithAttachedContainerForId(8, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(9, rp.id)) + } + test("SPARK-33288: multiple resource profiles") { when(podOperations .withField("status.phase", "Pending")) @@ -330,6 +473,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Target 1 executor for default profile, 2 for other profile, // make sure it's requested, even with an empty initial snapshot. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 2)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations).create(podWithAttachedContainerForId(1, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(2, rp.id)) verify(podOperations).create(podWithAttachedContainerForId(3, rp.id)) @@ -339,6 +483,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(runningExecutor(2, rp.id)) snapshotsStore.updatePod(runningExecutor(3, rp.id)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(3)).create(any()) verify(podOperations, never()).delete() @@ -346,6 +491,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // make sure all are requested. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 4, rp -> 3)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 4) verify(podOperations).create(podWithAttachedContainerForId(4, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(5, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(6, defaultProfile.id)) @@ -356,6 +502,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(pendingExecutor(5, defaultProfile.id)) snapshotsStore.updatePod(pendingExecutor(7, rp.id)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations, times(7)).create(any()) verify(podOperations, never()).delete() @@ -364,6 +511,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.advance(executorIdleTimeout * 2) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(7)).create(any()) verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "5", "6") verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "7") @@ -379,6 +527,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(deletedExecutor(7)) snapshotsStore.removeDeletedExecutors() snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) assert(!podsAllocatorUnderTest.isDeleted("5")) assert(!podsAllocatorUnderTest.isDeleted("6")) assert(!podsAllocatorUnderTest.isDeleted("7")) @@ -399,6 +548,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { .thenReturn(podOperations) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 6)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) // Initial request of pods verify(podOperations).create(podWithAttachedContainerForId(1)) verify(podOperations).create(podWithAttachedContainerForId(2)) @@ -414,6 +564,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // We move forward one allocation cycle waitForExecutorPodsClock.setTime(podAllocationDelay + 1) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) // We request pod 6 verify(podOperations).create(podWithAttachedContainerForId(6)) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 861d41cc50ac5..e4a73e24c3921 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -127,7 +127,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn test("Start all components") { schedulerBackendUnderTest.start() verify(podAllocator).setTotalExpectedExecutors(Map(defaultProfile -> 3)) - verify(podAllocator).start(TEST_SPARK_APP_ID) + verify(podAllocator).start(TEST_SPARK_APP_ID, schedulerBackendUnderTest) verify(lifecycleEventHandler).start(schedulerBackendUnderTest) verify(watchEvents).start(TEST_SPARK_APP_ID) verify(pollEvents).start(TEST_SPARK_APP_ID) From b8b6f884d28af05be21cc65080a7ed7d45b103b5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 3 Apr 2021 00:00:17 -0700 Subject: [PATCH 002/169] [SPARK-34948][K8S] Add ownerReference to executor configmap to fix leakages This PR aims to add `ownerReference` to the executor ConfigMap to fix leakage. SPARK-30985 maintains the executor config map explicitly inside Spark. However, this config map can be leaked when Spark drivers die accidentally or are killed by K8s. We need to add `ownerReference` to make K8s do the garbage collection these automatically. The number of ConfigMap is one of the resource quota. So, the leaked configMaps currently cause Spark jobs submission failures. No. Pass the CIs and check manually. K8s IT is tested manually. ``` KubernetesSuite: - Run SparkPi with no resources - Run SparkPi with a very long application name. - Use SparkLauncher.NO_RESOURCE - Run SparkPi with a master URL without a scheme. - Run SparkPi with an argument. - Run SparkPi with custom labels, annotations, and environment variables. - All pods have the same service account by default - Run extraJVMOptions check on driver - Run SparkRemoteFileTest using a remote data file - Verify logging configuration is picked from the provided SPARK_CONF_DIR/log4j.properties - Run SparkPi with env and mount secrets. - Run PySpark on simple pi.py example - Run PySpark to test a pyfiles example - Run PySpark with memory customization - Run in client mode. - Start pod creation from template - PVs with local storage - Launcher client dependencies - SPARK-33615: Launcher client archives - SPARK-33748: Launcher python client respecting PYSPARK_PYTHON - SPARK-33748: Launcher python client respecting spark.pyspark.python and spark.pyspark.driver.python - Launcher python client dependencies using a zip file - Test basic decommissioning - Test basic decommissioning with shuffle cleanup - Test decommissioning with dynamic allocation & shuffle cleanups - Test decommissioning timeouts - Run SparkR on simple dataframe.R example Run completed in 19 minutes, 2 seconds. Total number of tests run: 27 Suites: completed 2, aborted 0 Tests: succeeded 27, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` **BEFORE** ``` $ k get cm spark-exec-450b417895b3b2c7-conf-map -oyaml | grep ownerReferences ``` **AFTER** ``` $ k get cm spark-exec-bb37a27895b1c26c-conf-map -oyaml | grep ownerReferences f:ownerReferences: ``` Closes #32042 from dongjoon-hyun/SPARK-34948. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit a42dc93a2abf9490d68146b3586aec7fe2f9c102) Signed-off-by: Dongjoon Hyun --- .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 2 +- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 7 +++++-- .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 1 + 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 5fc81a6d84273..5ebd172f7dec6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -70,7 +70,7 @@ private[spark] class ExecutorPodsAllocator( private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) - private val driverPod = kubernetesDriverPodName + val driverPod = kubernetesDriverPodName .map(name => Option(kubernetesClient.pods() .withName(name) .get()) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index d58e38ab9794a..887afca05cb1c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -20,11 +20,13 @@ import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import scala.concurrent.Future +import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.submit.KubernetesClientUtils import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO @@ -67,13 +69,14 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - private def setUpExecutorConfigMap(): Unit = { + private def setUpExecutorConfigMap(driverPod: Option[Pod]): Unit = { val configMapName = KubernetesClientUtils.configMapNameExecutor val confFilesMap = KubernetesClientUtils .buildSparkConfDirFilesMap(configMapName, conf, Map.empty) val labels = Map(SPARK_APP_ID_LABEL -> applicationId(), SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) val configMap = KubernetesClientUtils.buildConfigMap(configMapName, confFilesMap, labels) + KubernetesUtils.addOwnerReference(driverPod.orNull, Seq(configMap)) kubernetesClient.configMaps().create(configMap) } @@ -96,7 +99,7 @@ private[spark] class KubernetesClusterSchedulerBackend( podAllocator.start(applicationId(), this) watchEvents.start(applicationId()) pollEvents.start(applicationId()) - setUpExecutorConfigMap() + setUpExecutorConfigMap(podAllocator.driverPod) } override def stop(): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index e4a73e24c3921..3573ffc07d2c8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -112,6 +112,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn .thenReturn(driverEndpointRef) when(kubernetesClient.pods()).thenReturn(podOperations) when(kubernetesClient.configMaps()).thenReturn(configMapsOperations) + when(podAllocator.driverPod).thenReturn(None) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( taskScheduler, sc, From e852a3c88d12ca96057f0f316940f26271ad1b2c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 21 May 2021 08:27:49 -0700 Subject: [PATCH 003/169] [SPARK-35482][K8S] Use `spark.blockManager.port` not the wrong `spark.blockmanager.port` in BasicExecutorFeatureStep ### What changes were proposed in this pull request? most spark conf keys are case sensitive, including `spark.blockManager.port`, we can not get the correct port number with `spark.blockmanager.port`. This PR changes the wrong key to `spark.blockManager.port` in `BasicExecutorFeatureStep`. This PR also ensures a fast fail when the port value is invalid for executor containers. When 0 is specified(it is valid as random port, but invalid as a k8s request), it should not be put in the `containerPort` field of executor pod desc. We do not expect executor pods to continuously fail to create because of invalid requests. ### Why are the changes needed? bugfix ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? new tests Closes #32621 from yaooqinn/SPARK-35482. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit d957426351149dd1b4e1106d1230f395934f61d2) Signed-off-by: Dongjoon Hyun --- .../features/BasicExecutorFeatureStep.scala | 24 +++++++++------ .../BasicExecutorFeatureStepSuite.scala | 29 +++++++++++++++++++ 2 files changed, 44 insertions(+), 9 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 250dd8238d9ea..a0a17cecf9a8e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -44,7 +44,10 @@ private[spark] class BasicExecutorFeatureStep( .getOrElse(throw new SparkException("Must specify the executor container image")) private val blockManagerPort = kubernetesConf .sparkConf - .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + .getInt(BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + + require(blockManagerPort == 0 || (1024 <= blockManagerPort && blockManagerPort < 65536), + "port number must be 0 or in [1024, 65535]") private val executorPodNamePrefix = kubernetesConf.resourceNamePrefix @@ -171,14 +174,17 @@ private[spark] class BasicExecutorFeatureStep( .replaceAll(ENV_EXECUTOR_ID, kubernetesConf.executorId)) } - val requiredPorts = Seq( - (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) - .map { case (name, port) => - new ContainerPortBuilder() - .withName(name) - .withContainerPort(port) - .build() - } + // 0 is invalid as kubernetes containerPort request, we shall leave it unmounted + val requiredPorts = if (blockManagerPort != 0) { + Seq( + (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) + .map { case (name, port) => + new ContainerPortBuilder() + .withName(name) + .withContainerPort(port) + .build() + } + } else Nil if (!isDefaultProfile) { if (pod.container != null && pod.container.getResources() != null) { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 66ece81aca646..df4693f313917 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -330,6 +330,35 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { SecretVolumeUtils.podHasVolume(podConfigured.pod, SPARK_CONF_VOLUME_EXEC) } + test("SPARK-35482: user correct block manager port for executor pods") { + try { + val initPod = SparkPod.initialPod() + val sm = new SecurityManager(baseConf) + val step1 = + new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + val containerPort1 = step1.configurePod(initPod).container.getPorts.get(0) + assert(containerPort1.getContainerPort === DEFAULT_BLOCKMANAGER_PORT, + s"should use port no. $DEFAULT_BLOCKMANAGER_PORT as default") + + baseConf.set(BLOCK_MANAGER_PORT, 12345) + val step2 = new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + val containerPort2 = step2.configurePod(initPod).container.getPorts.get(0) + assert(containerPort2.getContainerPort === 12345) + + baseConf.set(BLOCK_MANAGER_PORT, 1000) + val e = intercept[IllegalArgumentException] { + new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + } + assert(e.getMessage.contains("port number must be 0 or in [1024, 65535]")) + + baseConf.set(BLOCK_MANAGER_PORT, 0) + val step3 = new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + assert(step3.configurePod(initPod).container.getPorts.isEmpty, "random port") + } finally { + baseConf.remove(BLOCK_MANAGER_PORT) + } + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) From 5625c45ec9a60da763da4fb7ebdbf953ecb9abac Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 23 May 2021 08:07:57 -0700 Subject: [PATCH 004/169] [SPARK-35493][K8S] make `spark.blockManager.port` fallback for `spark.driver.blockManager.port` as same as other cluster managers ### What changes were proposed in this pull request? `spark.blockManager.port` does not work for k8s driver pods now, we should make it work as other cluster managers. ### Why are the changes needed? `spark.blockManager.port` should be able to work for spark driver pod ### Does this PR introduce _any_ user-facing change? yes, `spark.blockManager.port` will be respect iff it is present && `spark.driver.blockManager.port` is absent ### How was this patch tested? new tests Closes #32639 from yaooqinn/SPARK-35493. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit 96b0548ab6d5fe36833812f7b6424c984f75c6dd) Signed-off-by: Dongjoon Hyun --- .../k8s/features/BasicDriverFeatureStep.scala | 2 +- .../BasicDriverFeatureStepSuite.scala | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index cec8272beed57..7f34f30d59982 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -96,7 +96,7 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) val driverPort = conf.sparkConf.getInt(DRIVER_PORT.key, DEFAULT_DRIVER_PORT) val driverBlockManagerPort = conf.sparkConf.getInt( DRIVER_BLOCK_MANAGER_PORT.key, - DEFAULT_BLOCKMANAGER_PORT + conf.sparkConf.getInt(BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) ) val driverUIPort = SparkUI.getUIPort(conf.sparkConf) val driverContainer = new ContainerBuilder(pod.container) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 858b4f1494b8e..f0843225ea6e2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -213,6 +213,25 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { } } + test("SPARK-35493: make spark.blockManager.port be able to be fallen back to in driver pod") { + val initPod = SparkPod.initialPod() + val sparkConf = new SparkConf() + .set(CONTAINER_IMAGE, "spark-driver:latest") + .set(BLOCK_MANAGER_PORT, 1234) + val driverConf1 = KubernetesTestConf.createDriverConf(sparkConf) + val pod1 = new BasicDriverFeatureStep(driverConf1).configurePod(initPod) + val portMap1 = + pod1.container.getPorts.asScala.map { cp => (cp.getName -> cp.getContainerPort) }.toMap + assert(portMap1(BLOCK_MANAGER_PORT_NAME) === 1234, s"fallback to $BLOCK_MANAGER_PORT.key") + + val driverConf2 = + KubernetesTestConf.createDriverConf(sparkConf.set(DRIVER_BLOCK_MANAGER_PORT, 1235)) + val pod2 = new BasicDriverFeatureStep(driverConf2).configurePod(initPod) + val portMap2 = + pod2.container.getPorts.asScala.map { cp => (cp.getName -> cp.getContainerPort) }.toMap + assert(portMap2(BLOCK_MANAGER_PORT_NAME) === 1235) + } + def containerPort(name: String, portNumber: Int): ContainerPort = new ContainerPortBuilder() .withName(name) From 7c3e41121a921d0ff54dab1e9422adccdac4b9aa Mon Sep 17 00:00:00 2001 From: Chris Wu Date: Fri, 4 Jun 2021 06:59:49 -0700 Subject: [PATCH 005/169] [SPARK-32975][K8S] Add config for driver readiness timeout before executors start ### What changes were proposed in this pull request? Add a new config that controls the timeout of waiting for driver pod's readiness before allocating executor pods. This wait only happens once on application start. ### Why are the changes needed? The driver's headless service can be resolved by DNS only after the driver pod is ready. If the executor tries to connect to the headless service before driver pod is ready, it will hit UnkownHostException and get into error state but will not be restarted. **This case usually happens when the driver pod has sidecar containers but hasn't finished their creation when executors start.** So basically there is a race condition. This issue can be mitigated by tweaking this config. ### Does this PR introduce _any_ user-facing change? A new config `spark.kubernetes.allocation.driver.readinessTimeout` added. ### How was this patch tested? Exisiting tests. Closes #32752 from cchriswu/SPARK-32975-fix. Lead-authored-by: Chris Wu Co-authored-by: Chris Wu Signed-off-by: Dongjoon Hyun (cherry picked from commit 497c80a1ad7fdd605b75c8a6601fce35c7449578) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/deploy/k8s/Config.scala | 11 +++++++++++ .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 11 +++++++++++ .../cluster/k8s/ExecutorPodsAllocatorSuite.scala | 1 + 3 files changed, 23 insertions(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index fe92fae36c36c..de9d15850e1c8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -235,6 +235,17 @@ private[spark] object Config extends Logging { .checkValue(value => value > 0, "Allocation batch delay must be a positive time value.") .createWithDefaultString("1s") + val KUBERNETES_ALLOCATION_DRIVER_READINESS_TIMEOUT = + ConfigBuilder("spark.kubernetes.allocation.driver.readinessTimeout") + .doc("Time to wait for driver pod to get ready before creating executor pods. This wait " + + "only happens on application start. If timeout happens, executor pods will still be " + + "created.") + .version("3.1.3") + .timeConf(TimeUnit.SECONDS) + .checkValue(value => value > 0, "Allocation driver readiness timeout must be a positive " + + "time value.") + .createWithDefaultString("1s") + val KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT = ConfigBuilder("spark.kubernetes.allocation.executor.timeout") .doc("Time to wait before a newly created executor POD request, which does not reached " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 5ebd172f7dec6..358058e27a049 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s import java.time.Instant import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import scala.collection.JavaConverters._ @@ -61,6 +62,8 @@ private[spark] class ExecutorPodsAllocator( podAllocationDelay * 5, conf.get(KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT)) + private val driverPodReadinessTimeout = conf.get(KUBERNETES_ALLOCATION_DRIVER_READINESS_TIMEOUT) + private val executorIdleTimeout = conf.get(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT) * 1000 private val namespace = conf.get(KUBERNETES_NAMESPACE) @@ -99,6 +102,14 @@ private[spark] class ExecutorPodsAllocator( @volatile private var deletedExecutorIds = Set.empty[Long] def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { + // Wait until the driver pod is ready before starting executors, as the headless service won't + // be resolvable by DNS until the driver pod is ready. + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withName(kubernetesDriverPodName.get) + .waitUntilReady(driverPodReadinessTimeout, TimeUnit.SECONDS) + } snapshotsStore.addSubscriber(podAllocationDelay) { onNewSnapshots(applicationId, schedulerBackend, _) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 55be80ae29c7e..ed6ca2a68f9a7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -93,6 +93,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) when(driverPodOperations.get).thenReturn(driverPod) + when(driverPodOperations.waitUntilReady(any(), any())).thenReturn(driverPod) when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]), meq(secMgr), meq(kubernetesClient), any(classOf[ResourceProfile]))).thenAnswer(executorPodAnswer()) snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() From 38ec10632af272ef81e7c0359a5100e756aebcaf Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 10 Jun 2021 13:39:39 -0700 Subject: [PATCH 006/169] [SPARK-32975][K8S][FOLLOWUP] Avoid None.get exception ### What changes were proposed in this pull request? A follow-up for SPARK-32975 to avoid unexpected the `None.get` exception Run SparkPi with docker desktop, as podName is an option, we will got ```logtalk 21/06/09 01:09:12 ERROR Utils: Uncaught exception in thread main java.util.NoSuchElementException: None.get at scala.None$.get(Option.scala:529) at scala.None$.get(Option.scala:527) at org.apache.spark.scheduler.cluster.k8s.ExecutorPodsAllocator.$anonfun$start$1(ExecutorPodsAllocator.scala:110) at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1417) at org.apache.spark.scheduler.cluster.k8s.ExecutorPodsAllocator.start(ExecutorPodsAllocator.scala:111) at org.apache.spark.scheduler.cluster.k8s.KubernetesClusterSchedulerBackend.start(KubernetesClusterSchedulerBackend.scala:99) at org.apache.spark.scheduler.TaskSchedulerImpl.start(TaskSchedulerImpl.scala:220) at org.apache.spark.SparkContext.(SparkContext.scala:581) at org.apache.spark.SparkContext$.getOrCreate(SparkContext.scala:2686) at org.apache.spark.sql.SparkSession$Builder.$anonfun$getOrCreate$2(SparkSession.scala:948) at scala.Option.getOrElse(Option.scala:189) at org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:942) at org.apache.spark.examples.SparkPi$.main(SparkPi.scala:30) at org.apache.spark.examples.SparkPi.main(SparkPi.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:955) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1043) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1052) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) ``` ### Why are the changes needed? fix a regression ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Manual. Closes #32830 from yaooqinn/SPARK-32975. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit b4b78ce26567ce7ab83d47ce3b6af87c866bcacb) Signed-off-by: Dongjoon Hyun --- .../cluster/k8s/ExecutorPodsAllocator.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 358058e27a049..5429e36dda5ec 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -102,13 +102,15 @@ private[spark] class ExecutorPodsAllocator( @volatile private var deletedExecutorIds = Set.empty[Long] def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { - // Wait until the driver pod is ready before starting executors, as the headless service won't - // be resolvable by DNS until the driver pod is ready. - Utils.tryLogNonFatalError { - kubernetesClient - .pods() - .withName(kubernetesDriverPodName.get) - .waitUntilReady(driverPodReadinessTimeout, TimeUnit.SECONDS) + driverPod.foreach { pod => + // Wait until the driver pod is ready before starting executors, as the headless service won't + // be resolvable by DNS until the driver pod is ready. + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withName(pod.getMetadata.getName) + .waitUntilReady(driverPodReadinessTimeout, TimeUnit.SECONDS) + } } snapshotsStore.addSubscriber(podAllocationDelay) { onNewSnapshots(applicationId, schedulerBackend, _) From ee142c04df9452edb5b00d56475280c7e038dbf4 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 13 Jun 2021 09:11:14 -0700 Subject: [PATCH 007/169] [MINOR][K8S] Print the driver pod name instead of Some(name) if absent Print the driver pod name instead of Some(name) if absent fix error hint no new test Closes #32889 from yaooqinn/minork8s. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit 1125afd4622e6d3f7f14fca1ebcfebdfba6d9529) Signed-off-by: Dongjoon Hyun --- .../cluster/k8s/ExecutorPodsAllocator.scala | 2 +- .../cluster/k8s/ExecutorPodsAllocatorSuite.scala | 14 +++++++++++++- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 5429e36dda5ec..c83b8b854f298 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -78,7 +78,7 @@ private[spark] class ExecutorPodsAllocator( .withName(name) .get()) .getOrElse(throw new SparkException( - s"No pod was found named $kubernetesDriverPodName in the cluster in the " + + s"No pod was found named $name in the cluster in the " + s"namespace $namespace (this was supposed to be the driver pod.)."))) // Executor IDs that have been requested from Kubernetes but have not been detected in any diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index ed6ca2a68f9a7..7ec17298d902e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -28,7 +28,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesExecutorSpec} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -570,6 +570,18 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { verify(podOperations).create(podWithAttachedContainerForId(6)) } + test("print the pod name instead of Some(name) if pod is absent") { + val nonexistentPod = "i-do-not-exist" + val conf = new SparkConf().set(KUBERNETES_DRIVER_POD_NAME, nonexistentPod) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(nonexistentPod)).thenReturn(driverPodOperations) + when(driverPodOperations.get()).thenReturn(null) + val e = intercept[SparkException](new ExecutorPodsAllocator( + conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock)) + assert(e.getMessage.contains("No pod was found named i-do-not-exist in the cluster in the" + + " namespace default")) + } + private def executorPodAnswer(): Answer[KubernetesExecutorSpec] = (invocation: InvocationOnMock) => { val k8sConf: KubernetesExecutorConf = invocation.getArgument(0) From fd14c3074d13352cc2b558bb9b48df9b9e9ef757 Mon Sep 17 00:00:00 2001 From: Weiwei Yang Date: Tue, 19 Oct 2021 22:42:06 -0700 Subject: [PATCH 008/169] [SPARK-37049][K8S] executorIdleTimeout should check `creationTimestamp` instead of `startTime` SPARK-33099 added the support to respect `spark.dynamicAllocation.executorIdleTimeout` in `ExecutorPodsAllocator`. However, when it checks if a pending executor pod is timed out, it checks against the pod's [startTime](https://github.com/kubernetes/api/blob/2a5dae08c42b1e8fdc1379432d8898efece65363/core/v1/types.go#L3664-L3667), see code [here](https://github.com/apache/spark/blob/c2ba498ff678ddda034cedf45cc17fbeefe922fd/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala#L459). A pending pod `startTime` is empty, and this causes the function `isExecutorIdleTimedOut()` always return true for pending pods. This can be reproduced locally, run the following job ``` ${SPARK_HOME}/bin/spark-submit --master k8s://http://localhost:8001 --deploy-mode cluster --name spark-group-example \ --master k8s://http://localhost:8001 --deploy-mode cluster \ --class org.apache.spark.examples.GroupByTest \ --conf spark.executor.instances=1 \ --conf spark.kubernetes.namespace=spark-test \ --conf spark.kubernetes.executor.request.cores=1 \ --conf spark.dynamicAllocation.enabled=true \ --conf spark.shuffle.service.enabled=true \ --conf spark.dynamicAllocation.shuffleTracking.enabled=true \ --conf spark.shuffle.service.enabled=false \ --conf spark.kubernetes.container.image=local/spark:3.3.0 \ --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark \ local:///opt/spark/examples/jars/spark-examples_2.12-3.3.0-SNAPSHOT.jar \ 1000 1000 100 1000 ``` the local cluster doesn't have enough resources to run more than 4 executors, the rest of the executor pods will be pending. The job will have task backlogs and triggers to request more executors from K8s: ``` 21/10/19 22:51:45 INFO ExecutorPodsAllocator: Going to request 1 executors from Kubernetes for ResourceProfile Id: 0, target: 1 running: 0. 21/10/19 22:51:51 INFO ExecutorPodsAllocator: Going to request 1 executors from Kubernetes for ResourceProfile Id: 0, target: 2 running: 1. 21/10/19 22:51:52 INFO ExecutorPodsAllocator: Going to request 2 executors from Kubernetes for ResourceProfile Id: 0, target: 4 running: 2. 21/10/19 22:51:53 INFO ExecutorPodsAllocator: Going to request 4 executors from Kubernetes for ResourceProfile Id: 0, target: 8 running: 4. ... 21/10/19 22:52:14 INFO ExecutorPodsAllocator: Deleting 39 excess pod requests (23,59,32,41,50,68,35,44,17,8,53,62,26,71,11,56,29,38,47,20,65,5,14,46,64,73,55,49,40,67,58,13,22,31,7,16,52,70,43). 21/10/19 22:52:18 INFO ExecutorPodsAllocator: Deleting 28 excess pod requests (25,34,61,37,10,19,28,60,69,63,45,54,72,36,18,9,27,21,57,12,48,30,39,66,15,42,24,33). ``` At `22:51:45`, it starts to request executors; and at `22:52:14` it starts to delete excess executor pods. This is 29s but spark.dynamicAllocation.executorIdleTimeout is set to 60s. The config was not honored. ### What changes were proposed in this pull request? Change the check from using pod's `startTime` to `creationTimestamp`. [creationTimestamp](https://github.com/kubernetes/apimachinery/blob/e6c90c4366be1504309a6aafe0d816856450f36a/pkg/apis/meta/v1/types.go#L193-L201) is the timestamp when a pod gets created on K8s: ``` // CreationTimestamp is a timestamp representing the server time when this object was // created. It is not guaranteed to be set in happens-before order across separate operations. // Clients may not set this value. It is represented in RFC3339 form and is in UTC. ``` [startTime](https://github.com/kubernetes/api/blob/2a5dae08c42b1e8fdc1379432d8898efece65363/core/v1/types.go#L3664-L3667) is the timestamp when pod gets started: ``` // RFC 3339 date and time at which the object was acknowledged by the Kubelet. // This is before the Kubelet pulled the container image(s) for the pod. // +optional ``` a pending pod's startTime is empty. Here is a example of a pending pod: ``` NAMESPACE NAME READY STATUS RESTARTS AGE default pending-pod-example 0/1 Pending 0 2s kubectl get pod pending-pod-example -o yaml | grep creationTimestamp ---> creationTimestamp: "2021-10-19T16:17:52Z" // pending pod has no startTime kubectl get pod pending-pod-example -o yaml | grep startTime ---> // empty // running pod has startTime set to the timestamp when the pod gets started kubectl get pod coredns-558bd4d5db-6qrtx -n kube-system -o yaml | grep startTime f:startTime: {} ---> startTime: "2021-08-04T23:44:44Z" ``` ### Why are the changes needed? This fixed the issue that `spark.dynamicAllocation.executorIdleTimeout` currently is not honored by pending executor pods. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The PR includes the UT changes, that has the testing coverage for this issue. Closes #34319 from yangwwei/SPARK-37049. Authored-by: Weiwei Yang Signed-off-by: Dongjoon Hyun (cherry picked from commit 041cd5d7d15ec4184ae51a8a10a26bef05bd261f) Signed-off-by: Dongjoon Hyun --- .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 8 ++++---- .../cluster/k8s/ExecutorLifecycleTestUtils.scala | 4 +++- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index c83b8b854f298..9e1794fd3e8fc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -369,11 +369,11 @@ private[spark] class ExecutorPodsAllocator( private def isExecutorIdleTimedOut(state: ExecutorPodState, currentTime: Long): Boolean = { try { - val startTime = Instant.parse(state.pod.getStatus.getStartTime).toEpochMilli() - currentTime - startTime > executorIdleTimeout + val creationTime = Instant.parse(state.pod.getMetadata.getCreationTimestamp).toEpochMilli() + currentTime - creationTime > executorIdleTimeout } catch { - case _: Exception => - logDebug(s"Cannot get startTime of pod ${state.pod}") + case e: Exception => + logError(s"Cannot get the creationTimestamp of the pod: ${state.pod}", e) true } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index 41cba573d89c2..0b3ce6d7eb274 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -62,9 +62,11 @@ object ExecutorLifecycleTestUtils { def pendingExecutor(executorId: Long, rpId: Int = DEFAULT_RESOURCE_PROFILE_ID): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId, rpId)) + .editOrNewMetadata() + .withCreationTimestamp(Instant.now.toString) + .endMetadata() .editOrNewStatus() .withPhase("pending") - .withStartTime(Instant.now.toString) .endStatus() .build() } From 8e57cfeb9681626ff1887ed5113133cc3108a692 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 30 Nov 2021 18:41:18 -0800 Subject: [PATCH 009/169] [SPARK-37497][K8S] Promote `ExecutorPods[PollingSnapshot|WatchSnapshot]Source` to DeveloperApi ### What changes were proposed in this pull request? This PR aims to promote `ExecutorPodsWatchSnapshotSource` and `ExecutorPodsPollingSnapshotSource` as **stable** `DeveloperApi` in order to maintain it officially in a backward compatible way at Apache Spark 3.3.0. ### Why are the changes needed? - Since SPARK-24248 at Apache Spark 2.4.0, `ExecutorPodsWatchSnapshotSource` and `ExecutorPodsPollingSnapshotSource` have been used to monitor executor pods without any interface changes for over 3 years. - Apache Spark 3.1.1 makes `Kubernetes` module GA and provides an extensible external cluster manager framework. New `ExternalClusterManager` for K8s environment need to depend on this to monitor pods. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. Closes #34751 from dongjoon-hyun/SPARK-37497. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 2b044962cd6eff5a3a76f2808ee93b40bdf931df) Signed-off-by: Dongjoon Hyun --- .../k8s/ExecutorPodsPollingSnapshotSource.scala | 13 ++++++++++++- .../k8s/ExecutorPodsWatchSnapshotSource.scala | 14 +++++++++++++- 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala index da7fe7cdda328..6fcb87655b56d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala @@ -22,12 +22,21 @@ import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ import org.apache.spark.SparkConf +import org.apache.spark.annotation.{DeveloperApi, Since, Stable} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} -private[spark] class ExecutorPodsPollingSnapshotSource( +/** + * :: DeveloperApi :: + * + * A class used for polling K8s executor pods by ExternalClusterManagers. + * @since 3.1.3 + */ +@Stable +@DeveloperApi +class ExecutorPodsPollingSnapshotSource( conf: SparkConf, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, @@ -37,6 +46,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( private var pollingFuture: Future[_] = _ + @Since("3.1.3") def start(applicationId: String): Unit = { require(pollingFuture == null, "Cannot start polling more than once.") logDebug(s"Starting to check for executor pod state every $pollingInterval ms.") @@ -44,6 +54,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( new PollRunnable(applicationId), pollingInterval, pollingInterval, TimeUnit.MILLISECONDS) } + @Since("3.1.3") def stop(): Unit = { if (pollingFuture != null) { pollingFuture.cancel(true) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala index a6749a644e00c..7ac70b589c698 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala @@ -22,16 +22,27 @@ import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import org.apache.spark.annotation.{DeveloperApi, Since, Stable} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[spark] class ExecutorPodsWatchSnapshotSource( +/** + * :: DeveloperApi :: + * + * A class used for watching K8s executor pods by ExternalClusterManagers. + * + * @since 3.1.3 + */ +@Stable +@DeveloperApi +class ExecutorPodsWatchSnapshotSource( snapshotsStore: ExecutorPodsSnapshotsStore, kubernetesClient: KubernetesClient) extends Logging { private var watchConnection: Closeable = _ + @Since("3.1.3") def start(applicationId: String): Unit = { require(watchConnection == null, "Cannot start the watcher twice.") logDebug(s"Starting watch for pods with labels $SPARK_APP_ID_LABEL=$applicationId," + @@ -42,6 +53,7 @@ private[spark] class ExecutorPodsWatchSnapshotSource( .watch(new ExecutorPodsWatcher()) } + @Since("3.1.3") def stop(): Unit = { if (watchConnection != null) { Utils.tryLogNonFatalError { From 3c35c388fb929dffc234a6726757d9ae8e9e24df Mon Sep 17 00:00:00 2001 From: Sumeet Gajjar Date: Mon, 5 Apr 2021 17:32:43 -0500 Subject: [PATCH 010/169] [SPARK-34949][CORE] Prevent BlockManager reregister when Executor is shutting down ### What changes were proposed in this pull request? This PR prevents reregistering BlockManager when a Executor is shutting down. It is achieved by checking `executorShutdown` before calling `env.blockManager.reregister()`. ### Why are the changes needed? This change is required since Spark reports executors as active, even they are removed. I was testing Dynamic Allocation on K8s with about 300 executors. While doing so, when the executors were torn down due to `spark.dynamicAllocation.executorIdleTimeout`, I noticed all the executor pods being removed from K8s, however, under the "Executors" tab in SparkUI, I could see some executors listed as alive. [spark.sparkContext.statusTracker.getExecutorInfos.length](https://github.com/apache/spark/blob/65da9287bc5112564836a555cd2967fc6b05856f/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala#L105) also returned a value greater than 1. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a new test. ## Logs Following are the logs of the executor(Id:303) which re-registers `BlockManager` ``` 21/04/02 21:33:28 INFO CoarseGrainedExecutorBackend: Got assigned task 1076 21/04/02 21:33:28 INFO Executor: Running task 4.0 in stage 3.0 (TID 1076) 21/04/02 21:33:28 INFO MapOutputTrackerWorker: Updating epoch to 302 and clearing cache 21/04/02 21:33:28 INFO TorrentBroadcast: Started reading broadcast variable 3 21/04/02 21:33:28 INFO TransportClientFactory: Successfully created connection to /100.100.195.227:33703 after 76 ms (62 ms spent in bootstraps) 21/04/02 21:33:28 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 2.4 KB, free 168.0 MB) 21/04/02 21:33:28 INFO TorrentBroadcast: Reading broadcast variable 3 took 168 ms 21/04/02 21:33:28 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 3.9 KB, free 168.0 MB) 21/04/02 21:33:29 INFO MapOutputTrackerWorker: Don't have map outputs for shuffle 1, fetching them 21/04/02 21:33:29 INFO MapOutputTrackerWorker: Doing the fetch; tracker endpoint = NettyRpcEndpointRef(spark://MapOutputTrackerda-lite-test-4-7a57e478947d206d-driver-svc.dex-app-n5ttnbmg.svc:7078) 21/04/02 21:33:29 INFO MapOutputTrackerWorker: Got the output locations 21/04/02 21:33:29 INFO ShuffleBlockFetcherIterator: Getting 2 non-empty blocks including 1 local blocks and 1 remote blocks 21/04/02 21:33:30 INFO TransportClientFactory: Successfully created connection to /100.100.80.103:40971 after 660 ms (528 ms spent in bootstraps) 21/04/02 21:33:30 INFO ShuffleBlockFetcherIterator: Started 1 remote fetches in 1042 ms 21/04/02 21:33:31 INFO Executor: Finished task 4.0 in stage 3.0 (TID 1076). 1276 bytes result sent to driver . . . 21/04/02 21:34:16 INFO CoarseGrainedExecutorBackend: Driver commanded a shutdown 21/04/02 21:34:16 INFO Executor: Told to re-register on heartbeat 21/04/02 21:34:16 INFO BlockManager: BlockManager BlockManagerId(303, 100.100.122.34, 41265, None) re-registering with master 21/04/02 21:34:16 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(303, 100.100.122.34, 41265, None) 21/04/02 21:34:16 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(303, 100.100.122.34, 41265, None) 21/04/02 21:34:16 INFO BlockManager: Reporting 0 blocks to the master. 21/04/02 21:34:16 INFO MemoryStore: MemoryStore cleared 21/04/02 21:34:16 INFO BlockManager: BlockManager stopped 21/04/02 21:34:16 INFO FileDataSink: Closing sink with output file = /tmp/safari-events/.des_analysis/safari-events/hdp_spark_monitoring_random-container-037caf27-6c77-433f-820f-03cd9c7d9b6e-spark-8a492407d60b401bbf4309a14ea02ca2_events.tsv 21/04/02 21:34:16 INFO HonestProfilerBasedThreadSnapshotProvider: Stopping agent 21/04/02 21:34:16 INFO HonestProfilerHandler: Stopping honest profiler agent 21/04/02 21:34:17 INFO ShutdownHookManager: Shutdown hook called 21/04/02 21:34:17 INFO ShutdownHookManager: Deleting directory /var/data/spark-d886588c-2a7e-491d-bbcb-4f58b3e31001/spark-4aa337a0-60c0-45da-9562-8c50eaff3cea ``` Closes #32043 from sumeetgajjar/SPARK-34949. Authored-by: Sumeet Gajjar Signed-off-by: Mridul Muralidharan gmail.com> (cherry picked from commit a9ca1978ae8ecc53e2ef9e14b4be70dc8f5d9341) Signed-off-by: Mridul Muralidharan --- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/executor/ExecutorSuite.scala | 66 ++++++++++++++----- 2 files changed, 52 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e7f1b8f3cf17a..4ead4397e9739 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -995,7 +995,7 @@ private[spark] class Executor( try { val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( message, new RpcTimeout(HEARTBEAT_INTERVAL_MS.millis, EXECUTOR_HEARTBEAT_INTERVAL.key)) - if (response.reregisterBlockManager) { + if (!executorShutdown.get && response.reregisterBlockManager) { logInfo("Told to re-register on heartbeat") env.blockManager.reregister() } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 97ffb36062dbc..a237447b0fa2d 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -270,6 +270,17 @@ class ExecutorSuite extends SparkFunSuite heartbeatZeroAccumulatorUpdateTest(false) } + private def withMockHeartbeatReceiverRef(executor: Executor) + (func: RpcEndpointRef => Unit): Unit = { + val executorClass = classOf[Executor] + val mockReceiverRef = mock[RpcEndpointRef] + val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") + receiverRef.setAccessible(true) + receiverRef.set(executor, mockReceiverRef) + + func(mockReceiverRef) + } + private def withHeartbeatExecutor(confs: (String, String)*) (f: (Executor, ArrayBuffer[Heartbeat]) => Unit): Unit = { val conf = new SparkConf @@ -277,22 +288,18 @@ class ExecutorSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) withExecutor("id", "localhost", SparkEnv.get) { executor => - val executorClass = classOf[Executor] - - // Save all heartbeats sent into an ArrayBuffer for verification - val heartbeats = ArrayBuffer[Heartbeat]() - val mockReceiver = mock[RpcEndpointRef] - when(mockReceiver.askSync(any[Heartbeat], any[RpcTimeout])(any)) - .thenAnswer((invocation: InvocationOnMock) => { - val args = invocation.getArguments() - heartbeats += args(0).asInstanceOf[Heartbeat] - HeartbeatResponse(false) - }) - val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") - receiverRef.setAccessible(true) - receiverRef.set(executor, mockReceiver) + withMockHeartbeatReceiverRef(executor) { mockReceiverRef => + // Save all heartbeats sent into an ArrayBuffer for verification + val heartbeats = ArrayBuffer[Heartbeat]() + when(mockReceiverRef.askSync(any[Heartbeat], any[RpcTimeout])(any)) + .thenAnswer((invocation: InvocationOnMock) => { + val args = invocation.getArguments() + heartbeats += args(0).asInstanceOf[Heartbeat] + HeartbeatResponse(false) + }) - f(executor, heartbeats) + f(executor, heartbeats) + } } } @@ -416,6 +423,35 @@ class ExecutorSuite extends SparkFunSuite assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) } + test("SPARK-34949: do not re-register BlockManager when executor is shutting down") { + val reregisterInvoked = new AtomicBoolean(false) + val mockBlockManager = mock[BlockManager] + when(mockBlockManager.reregister()).thenAnswer { (_: InvocationOnMock) => + reregisterInvoked.getAndSet(true) + } + val conf = new SparkConf(false).setAppName("test").setMaster("local[2]") + val mockEnv = createMockEnv(conf, new JavaSerializer(conf)) + when(mockEnv.blockManager).thenReturn(mockBlockManager) + + withExecutor("id", "localhost", mockEnv) { executor => + withMockHeartbeatReceiverRef(executor) { mockReceiverRef => + when(mockReceiverRef.askSync(any[Heartbeat], any[RpcTimeout])(any)).thenAnswer { + (_: InvocationOnMock) => HeartbeatResponse(reregisterBlockManager = true) + } + val reportHeartbeat = PrivateMethod[Unit](Symbol("reportHeartBeat")) + executor.invokePrivate(reportHeartbeat()) + assert(reregisterInvoked.get(), "BlockManager.reregister should be invoked " + + "on HeartbeatResponse(reregisterBlockManager = true) when executor is not shutting down") + + reregisterInvoked.getAndSet(false) + executor.stop() + executor.invokePrivate(reportHeartbeat()) + assert(!reregisterInvoked.get(), + "BlockManager.reregister should not be invoked when executor is shutting down") + } + } + } + test("SPARK-33587: isFatalError") { def errorInThreadPool(e: => Throwable): Throwable = { intercept[Throwable] { From c3523986c7091627642f66641ff63680b88021bb Mon Sep 17 00:00:00 2001 From: skotlov Date: Wed, 21 Apr 2021 22:54:16 -0700 Subject: [PATCH 011/169] [SPARK-34674][CORE][K8S] Close SparkContext after the Main method has finished ### What changes were proposed in this pull request? Close SparkContext after the Main method has finished, to allow SparkApplication on K8S to complete. This is fixed version of [merged and reverted PR](https://github.com/apache/spark/pull/32081). ### Why are the changes needed? if I don't call the method sparkContext.stop() explicitly, then a Spark driver process doesn't terminate even after its Main method has been completed. This behaviour is different from spark on yarn, where the manual sparkContext stopping is not required. It looks like, the problem is in using non-daemon threads, which prevent the driver jvm process from terminating. So I have inserted code that closes sparkContext automatically. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually on the production AWS EKS environment in my company. Closes #32283 from kotlovs/close-spark-context-on-exit-2. Authored-by: skotlov Signed-off-by: Dongjoon Hyun (cherry picked from commit b17a0e6931cac98cc839c047b1b5d4ea6d052009) Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index acdddbcb89401..bc4d677c640af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -956,6 +956,15 @@ private[spark] class SparkSubmit extends Logging { } catch { case t: Throwable => throw findCause(t) + } finally { + if (!isShell(args.primaryResource) && !isSqlShell(args.mainClass) && + !isThriftServer(args.mainClass)) { + try { + SparkContext.getActive.foreach(_.stop()) + } catch { + case e: Throwable => logError(s"Failed to close SparkContext: $e") + } + } } } From a004fb601aa45ca08d0dc31045193cde9aad8689 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 18 Jul 2021 22:26:23 -0700 Subject: [PATCH 012/169] [SPARK-36193][CORE] Recover SparkSubmit.runMain not to stop SparkContext in non-K8s env ### What changes were proposed in this pull request? According to the discussion on https://github.com/apache/spark/pull/32283 , this PR aims to limit the feature of SPARK-34674 to K8s environment only. ### Why are the changes needed? To reduce the behavior change in non-K8s environment. ### Does this PR introduce _any_ user-facing change? The change behavior is consistent with 3.1.1 and older Spark releases. ### How was this patch tested? N/A Closes #33403 from dongjoon-hyun/SPARK-36193. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit fd3e9ce0b9ee09c7dce9f2e029fe96eac51eab96) Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index bc4d677c640af..5a9c550fc2068 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -957,8 +957,8 @@ private[spark] class SparkSubmit extends Logging { case t: Throwable => throw findCause(t) } finally { - if (!isShell(args.primaryResource) && !isSqlShell(args.mainClass) && - !isThriftServer(args.mainClass)) { + if (args.master.startsWith("k8s") && !isShell(args.primaryResource) && + !isSqlShell(args.mainClass) && !isThriftServer(args.mainClass)) { try { SparkContext.getActive.foreach(_.stop()) } catch { From a92ed12f1e5144ed0e080160b7fb46375e71a9fb Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 26 Jun 2021 12:48:24 +0800 Subject: [PATCH 013/169] [SPARK-35879][CORE][SHUFFLE] Fix performance regression caused by collectFetchRequests ### What changes were proposed in this pull request? This PR fixes perf regression at the executor side when creating fetch requests with large initial partitions ![image](https://user-images.githubusercontent.com/8326978/123270865-dd21e800-d532-11eb-8447-ad80e47b034f.png) In NetEase, we had an online job that took `45min` to "fetch" about 100MB of shuffle data, which actually turned out that it was just collecting fetch requests slowly. Normally, such a task should finish in seconds. See the `DEBUG` log ``` 21/06/22 11:52:26 DEBUG BlockManagerStorageEndpoint: Sent response: 0 to kyuubi.163.org: 21/06/22 11:53:05 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 3941440 at BlockManagerId(12, .., 43559, None) with 19 blocks 21/06/22 11:53:44 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 3716400 at BlockManagerId(20, .., 38287, None) with 18 blocks 21/06/22 11:54:41 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 4559280 at BlockManagerId(6, .., 39689, None) with 22 blocks 21/06/22 11:55:08 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 3120160 at BlockManagerId(33, .., 39449, None) with 15 blocks ``` I also create a test case locally with my local laptop docker env to give some reproducible cases. ``` bin/spark-sql --conf spark.kubernetes.file.upload.path=./ --master k8s://https://kubernetes.docker.internal:6443 --conf spark.kubernetes.container.image=yaooqinn/spark:v20210624-5 -c spark.kubernetes.context=docker-for-desktop_1 --num-executors 5 --driver-memory 5g --conf spark.kubernetes.executor.podNamePrefix=sparksql ``` ```sql SET spark.sql.adaptive.enabled=true; SET spark.sql.shuffle.partitions=3000; SELECT /*+ REPARTITION */ 1 as pid, id from range(1, 1000000, 1, 500); SELECT /*+ REPARTITION(pid, id) */ 1 as pid, id from range(1, 1000000, 1, 500); ``` ### Why are the changes needed? fix perf regression which was introduced by SPARK-29292 (3ad4863673fc46080dda963be3055a3e554cfbc7) in v3.1.0. 3ad4863673fc46080dda963be3055a3e554cfbc7 is for support compilation with scala 2.13 but the performance losses is huge. We need to consider backporting this PR to branch 3.1. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Mannully, #### before ```log 21/06/23 13:54:22 DEBUG ShuffleBlockFetcherIterator: maxBytesInFlight: 50331648, targetRemoteRequestSize: 10066329, maxBlocksInFlightPerAddress: 2147483647 21/06/23 13:54:38 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 2314708 at BlockManagerId(2, 10.1.3.114, 36423, None) with 86 blocks 21/06/23 13:54:59 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 2636612 at BlockManagerId(3, 10.1.3.115, 34293, None) with 87 blocks 21/06/23 13:55:18 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 2508706 at BlockManagerId(4, 10.1.3.116, 41869, None) with 90 blocks 21/06/23 13:55:34 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 2350854 at BlockManagerId(5, 10.1.3.117, 45787, None) with 85 blocks 21/06/23 13:55:34 INFO ShuffleBlockFetcherIterator: Getting 438 (11.8 MiB) non-empty blocks including 90 (2.5 MiB) local and 0 (0.0 B) host-local and 348 (9.4 MiB) remote blocks 21/06/23 13:55:34 DEBUG ShuffleBlockFetcherIterator: Sending request for 87 blocks (2.5 MiB) from 10.1.3.115:34293 21/06/23 13:55:34 INFO TransportClientFactory: Successfully created connection to /10.1.3.115:34293 after 1 ms (0 ms spent in bootstraps) 21/06/23 13:55:34 DEBUG ShuffleBlockFetcherIterator: Sending request for 90 blocks (2.4 MiB) from 10.1.3.116:41869 21/06/23 13:55:34 INFO TransportClientFactory: Successfully created connection to /10.1.3.116:41869 after 2 ms (0 ms spent in bootstraps) 21/06/23 13:55:34 DEBUG ShuffleBlockFetcherIterator: Sending request for 85 blocks (2.2 MiB) from 10.1.3.117:45787 ``` ```log 21/06/23 14:00:45 INFO MapOutputTracker: Broadcast outputstatuses size = 411, actual size = 828997 21/06/23 14:00:45 INFO MapOutputTrackerWorker: Got the map output locations 21/06/23 14:00:45 DEBUG ShuffleBlockFetcherIterator: maxBytesInFlight: 50331648, targetRemoteRequestSize: 10066329, maxBlocksInFlightPerAddress: 2147483647 21/06/23 14:00:55 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 1894389 at BlockManagerId(2, 10.1.3.114, 36423, None) with 99 blocks 21/06/23 14:01:04 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 1919993 at BlockManagerId(3, 10.1.3.115, 34293, None) with 100 blocks 21/06/23 14:01:14 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 1977186 at BlockManagerId(5, 10.1.3.117, 45787, None) with 103 blocks 21/06/23 14:01:23 DEBUG ShuffleBlockFetcherIterator: Creating fetch request of 1938336 at BlockManagerId(4, 10.1.3.116, 41869, None) with 101 blocks 21/06/23 14:01:23 INFO ShuffleBlockFetcherIterator: Getting 500 (9.1 MiB) non-empty blocks including 97 (1820.3 KiB) local and 0 (0.0 B) host-local and 403 (7.4 MiB) remote blocks 21/06/23 14:01:23 DEBUG ShuffleBlockFetcherIterator: Sending request for 101 blocks (1892.9 KiB) from 10.1.3.116:41869 21/06/23 14:01:23 DEBUG ShuffleBlockFetcherIterator: Sending request for 103 blocks (1930.8 KiB) from 10.1.3.117:45787 21/06/23 14:01:23 DEBUG ShuffleBlockFetcherIterator: Sending request for 99 blocks (1850.0 KiB) from 10.1.3.114:36423 21/06/23 14:01:23 DEBUG ShuffleBlockFetcherIterator: Sending request for 100 blocks (1875.0 KiB) from 10.1.3.115:34293 21/06/23 14:01:23 INFO ShuffleBlockFetcherIterator: Started 4 remote fetches in 37889 ms ``` #### After ```log 21/06/24 13:01:16 DEBUG ShuffleBlockFetcherIterator: maxBytesInFlight: 50331648, targetRemoteRequestSize: 10066329, maxBlocksInFlightPerAddress: 2147483647 21/06/24 13:01:16 INFO ShuffleBlockFetcherIterator: ==> Call blockInfos.map(_._2).sum: 40 ms 21/06/24 13:01:16 INFO ShuffleBlockFetcherIterator: ==> Call mergeFetchBlockInfo for shuffle_0_9_2990_2997/9: 0 ms 21/06/24 13:01:16 INFO ShuffleBlockFetcherIterator: ==> Call mergeFetchBlockInfo for shuffle_0_15_2395_2997/15: 0 ms ``` Closes #33063 from yaooqinn/SPARK-35879. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit 14d4decf736297e2bf4d824ccbd604c9da49ccf4) Signed-off-by: Kent Yao --- .../storage/ShuffleBlockFetcherIterator.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index fa4e46590aa5e..0c37a5b21a870 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -317,7 +317,10 @@ final class ShuffleBlockFetcherIterator( hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum } else { remoteBlockBytes += blockInfos.map(_._2).sum - collectFetchRequests(address, blockInfos, collectedRemoteRequests) + val (_, timeCost) = Utils.timeTakenMs[Unit] { + collectFetchRequests(address, blockInfos, collectedRemoteRequests) + } + logDebug(s"Collected remote fetch requests for $address in $timeCost ms") } } val numRemoteBlocks = collectedRemoteRequests.map(_.blocks.size).sum @@ -345,10 +348,10 @@ final class ShuffleBlockFetcherIterator( curBlocks: Seq[FetchBlockInfo], address: BlockManagerId, isLast: Boolean, - collectedRemoteRequests: ArrayBuffer[FetchRequest]): Seq[FetchBlockInfo] = { + collectedRemoteRequests: ArrayBuffer[FetchRequest]): ArrayBuffer[FetchBlockInfo] = { val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks, doBatchFetch) numBlocksToFetch += mergedBlocks.size - var retBlocks = Seq.empty[FetchBlockInfo] + val retBlocks = new ArrayBuffer[FetchBlockInfo] if (mergedBlocks.length <= maxBlocksInFlightPerAddress) { collectedRemoteRequests += createFetchRequest(mergedBlocks, address) } else { @@ -358,7 +361,7 @@ final class ShuffleBlockFetcherIterator( } else { // The last group does not exceed `maxBlocksInFlightPerAddress`. Put it back // to `curBlocks`. - retBlocks = blocks + retBlocks ++= blocks numBlocksToFetch -= blocks.size } } @@ -372,26 +375,24 @@ final class ShuffleBlockFetcherIterator( collectedRemoteRequests: ArrayBuffer[FetchRequest]): Unit = { val iterator = blockInfos.iterator var curRequestSize = 0L - var curBlocks = Seq.empty[FetchBlockInfo] + var curBlocks = new ArrayBuffer[FetchBlockInfo]() while (iterator.hasNext) { val (blockId, size, mapIndex) = iterator.next() assertPositiveBlockSize(blockId, size) - curBlocks = curBlocks ++ Seq(FetchBlockInfo(blockId, size, mapIndex)) + curBlocks += FetchBlockInfo(blockId, size, mapIndex) curRequestSize += size // For batch fetch, the actual block in flight should count for merged block. val mayExceedsMaxBlocks = !doBatchFetch && curBlocks.size >= maxBlocksInFlightPerAddress if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) { - curBlocks = createFetchRequests(curBlocks, address, isLast = false, + curBlocks = createFetchRequests(curBlocks.toSeq, address, isLast = false, collectedRemoteRequests) curRequestSize = curBlocks.map(_.size).sum } } // Add in the final request if (curBlocks.nonEmpty) { - curBlocks = createFetchRequests(curBlocks, address, isLast = true, - collectedRemoteRequests) - curRequestSize = curBlocks.map(_.size).sum + createFetchRequests(curBlocks.toSeq, address, isLast = true, collectedRemoteRequests) } } @@ -889,7 +890,7 @@ object ShuffleBlockFetcherIterator { blocks: Seq[FetchBlockInfo], doBatchFetch: Boolean): Seq[FetchBlockInfo] = { val result = if (doBatchFetch) { - var curBlocks = new ArrayBuffer[FetchBlockInfo] + val curBlocks = new ArrayBuffer[FetchBlockInfo] val mergedBlockInfo = new ArrayBuffer[FetchBlockInfo] def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = { From d6b5c3931a6173cc1eb66af022c02242bee2985f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 22 Feb 2021 21:11:21 +0800 Subject: [PATCH 014/169] [SPARK-34473][SQL] Avoid NPE in DataFrameReader.schema(StructType) ### What changes were proposed in this pull request? This fixes a regression in `DataFrameReader.schema(StructType)`, to avoid NPE if the given `StructType` is null. Note that, passing null to Spark public APIs leads to undefined behavior. There is no document mentioning the null behavior, and it's just an accident that `DataFrameReader.schema(StructType)` worked before. So I think this is not a 3.1 blocker. ### Why are the changes needed? It fixes a 3.1 regression ### Does this PR introduce _any_ user-facing change? yea, now `df.read.schema(null: StructType)` is a noop as before, while in the current branch-3.1 it throws NPE. ### How was this patch tested? It's undefined behavior and is very obvious, so I didn't add a test. We should add tests when we clearly define and fix the null behavior for all public APIs. Closes #31593 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 02c784ca686fc675b63ce37f03215bc6c2fec869) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/DataFrameReader.scala | 11 +++++------ .../apache/spark/sql/streaming/DataStreamReader.scala | 11 +++++------ 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b94c42a2c9544..e4da076035171 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -73,8 +73,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def schema(schema: StructType): DataFrameReader = { - val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(replaced) + if (schema != null) { + val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] + this.userSpecifiedSchema = Option(replaced) + } this } @@ -90,10 +92,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 2.3.0 */ def schema(schemaString: String): DataFrameReader = { - val rawSchema = StructType.fromDDL(schemaString) - val schema = CharVarcharUtils.failIfHasCharVarchar(rawSchema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(schema) - this + schema(StructType.fromDDL(schemaString)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index d82fa9e88592f..06c75791ad5c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -64,8 +64,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * @since 2.0.0 */ def schema(schema: StructType): DataStreamReader = { - val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(replaced) + if (schema != null) { + val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] + this.userSpecifiedSchema = Option(replaced) + } this } @@ -77,10 +79,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * @since 2.3.0 */ def schema(schemaString: String): DataStreamReader = { - val rawSchema = StructType.fromDDL(schemaString) - val schema = CharVarcharUtils.failIfHasCharVarchar(rawSchema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(schema) - this + schema(StructType.fromDDL(schemaString)) } /** From 866e6839f53c5e8b7243e187111932180e0377e9 Mon Sep 17 00:00:00 2001 From: Linhong Liu Date: Tue, 23 Feb 2021 15:51:02 +0800 Subject: [PATCH 015/169] [SPARK-34490][SQL] Analysis should fail if the view refers a dropped table When resolving a view, we use the captured view name in `AnalysisContext` to distinguish whether a relation name is a view or a table. But if the resolution failed, other rules (e.g. `ResolveTables`) will try to resolve the relation again but without `AnalysisContext`. So, in this case, the resolution may be incorrect. For example, if the view refers to a dropped table while a view with the same name exists, the dropped table will be resolved as a view rather than an unresolved exception. bugfix no newly added test cases Closes #31606 from linhongliu-db/fix-temp-view-master. Lead-authored-by: Linhong Liu Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com> Signed-off-by: Wenchen Fan (cherry picked from commit be675a052c38a36ce5e33ba56bdc69cc8972b3e8) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 27 +++++++++++++------ .../analysis/TableLookupCacheSuite.scala | 13 ++++++--- .../sql/execution/SQLViewTestSuite.scala | 20 ++++++++++++++ 3 files changed, 49 insertions(+), 11 deletions(-) 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 e9e8ba842a36f..bf8003105ada6 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 @@ -872,16 +872,16 @@ class Analyzer(override val catalogManager: CatalogManager) object ResolveTempViews extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case u @ UnresolvedRelation(ident, _, isStreaming) => - lookupTempView(ident, isStreaming).getOrElse(u) + lookupTempView(ident, isStreaming, performCheck = true).getOrElse(u) case i @ InsertIntoStatement(UnresolvedRelation(ident, _, false), _, _, _, _, _) => - lookupTempView(ident) + lookupTempView(ident, performCheck = true) .map(view => i.copy(table = view)) .getOrElse(i) // TODO (SPARK-27484): handle streaming write commands when we have them. case write: V2WriteCommand => write.table match { case UnresolvedRelation(ident, _, false) => - lookupTempView(ident).map(EliminateSubqueryAliases(_)).map { + lookupTempView(ident, performCheck = true).map(EliminateSubqueryAliases(_)).map { case r: DataSourceV2Relation => write.withNewTable(r) case _ => throw new AnalysisException("Cannot write into temp view " + s"${ident.quoted} as it's not a data source v2 relation.") @@ -906,7 +906,9 @@ class Analyzer(override val catalogManager: CatalogManager) } def lookupTempView( - identifier: Seq[String], isStreaming: Boolean = false): Option[LogicalPlan] = { + identifier: Seq[String], + isStreaming: Boolean = false, + performCheck: Boolean = false): Option[LogicalPlan] = { // Permanent View can't refer to temp views, no need to lookup at all. if (isResolvingView && !referredTempViewNames.contains(identifier)) return None @@ -920,7 +922,7 @@ class Analyzer(override val catalogManager: CatalogManager) throw new AnalysisException(s"${identifier.quoted} is not a temp view of streaming " + s"logical plan, please use batch API such as `DataFrameReader.table` to read it.") } - tmpView.map(ResolveRelations.resolveViews) + tmpView.map(ResolveRelations.resolveViews(_, performCheck)) } } @@ -1074,7 +1076,7 @@ class Analyzer(override val catalogManager: CatalogManager) // look at `AnalysisContext.catalogAndNamespace` when resolving relations with single-part name. // If `AnalysisContext.catalogAndNamespace` is non-empty, analyzer will expand single-part names // with it, instead of current catalog and namespace. - def resolveViews(plan: LogicalPlan): LogicalPlan = plan match { + def resolveViews(plan: LogicalPlan, performCheck: Boolean = false): LogicalPlan = plan match { // The view's child should be a logical plan parsed from the `desc.viewText`, the variable // `viewText` should be defined, or else we throw an error on the generation of the View // operator. @@ -1093,9 +1095,18 @@ class Analyzer(override val catalogManager: CatalogManager) executeSameContext(child) } } + // Fail the analysis eagerly because outside AnalysisContext, the unresolved operators + // inside a view maybe resolved incorrectly. + // But for commands like `DropViewCommand`, resolving view is unnecessary even though + // there is unresolved node. So use the `performCheck` flag to skip the analysis check + // for these commands. + // TODO(SPARK-34504): avoid unnecessary view resolving and remove the `performCheck` flag + if (performCheck) { + checkAnalysis(newChild) + } view.copy(child = newChild) case p @ SubqueryAlias(_, view: View) => - p.copy(child = resolveViews(view)) + p.copy(child = resolveViews(view, performCheck)) case _ => plan } @@ -1133,7 +1144,7 @@ class Analyzer(override val catalogManager: CatalogManager) case u: UnresolvedRelation => lookupRelation(u.multipartIdentifier, u.options, u.isStreaming) - .map(resolveViews).getOrElse(u) + .map(resolveViews(_, performCheck = true)).getOrElse(u) case u @ UnresolvedTable(identifier, cmd) => lookupTableOrView(identifier).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala index 3e9a8b71a8fb6..ec9480514ba2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import java.io.File +import scala.collection.JavaConverters._ + import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -27,8 +29,8 @@ import org.scalatest.matchers.must.Matchers import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.connector.InMemoryTableCatalog -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, V1Table} +import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table} import org.apache.spark.sql.types._ class TableLookupCacheSuite extends AnalysisTest with Matchers { @@ -46,7 +48,12 @@ class TableLookupCacheSuite extends AnalysisTest with Matchers { ignoreIfExists = false) val v2Catalog = new InMemoryTableCatalog { override def loadTable(ident: Identifier): Table = { - V1Table(externalCatalog.getTable("default", ident.name)) + val catalogTable = externalCatalog.getTable("default", ident.name) + new InMemoryTable( + catalogTable.identifier.table, + catalogTable.schema, + Array.empty, + Map.empty[String, String].asJava) } override def name: String = CatalogManager.SESSION_CATALOG_NAME } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index 68e1a682562ac..84a20bb16ad86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -258,6 +258,26 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { checkViewOutput(viewName, Seq(Row(2))) } } + + test("SPARK-34490 - query should fail if the view refers a dropped table") { + withTable("t") { + Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t") + val viewName = createView("testView", "SELECT * FROM t") + withView(viewName) { + // Always create a temp view in this case, not use `createView` on purpose + sql("CREATE TEMP VIEW t AS SELECT 1 AS c1") + withTempView("t") { + checkViewOutput(viewName, Seq(Row(2), Row(3), Row(1))) + // Manually drop table `t` to see if the query will fail + sql("DROP TABLE IF EXISTS default.t") + val e = intercept[AnalysisException] { + sql(s"SELECT * FROM $viewName").collect() + }.getMessage + assert(e.contains("Table or view not found: t")) + } + } + } + } } class LocalTempViewTestSuite extends SQLViewTestSuite with SharedSparkSession { From 6003f7c015444a70293e9127c0e7929040f32d4a Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Wed, 24 Feb 2021 21:32:19 +0800 Subject: [PATCH 016/169] [SPARK-34515][SQL] Fix NPE if InSet contains null value during getPartitionsByFilter MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Skip null value during rewrite `InSet` to `>= and <=` at getPartitionsByFilter. ### Why are the changes needed? Spark will convert `InSet` to `>= and <=` if it's values size over `spark.sql.hive.metastorePartitionPruningInSetThreshold` during pruning partition . At this case, if values contain a null, we will get such exception    ``` java.lang.NullPointerException at org.apache.spark.unsafe.types.UTF8String.compareTo(UTF8String.java:1389) at org.apache.spark.unsafe.types.UTF8String.compareTo(UTF8String.java:50) at scala.math.LowPriorityOrderingImplicits$$anon$3.compare(Ordering.scala:153) at java.util.TimSort.countRunAndMakeAscending(TimSort.java:355) at java.util.TimSort.sort(TimSort.java:220) at java.util.Arrays.sort(Arrays.java:1438) at scala.collection.SeqLike.sorted(SeqLike.scala:659) at scala.collection.SeqLike.sorted$(SeqLike.scala:647) at scala.collection.AbstractSeq.sorted(Seq.scala:45) at org.apache.spark.sql.hive.client.Shim_v0_13.convert$1(HiveShim.scala:772) at org.apache.spark.sql.hive.client.Shim_v0_13.$anonfun$convertFilters$4(HiveShim.scala:826) at scala.collection.immutable.Stream.flatMap(Stream.scala:489) at org.apache.spark.sql.hive.client.Shim_v0_13.convertFilters(HiveShim.scala:826) at org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:848) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getPartitionsByFilter$1(HiveClientImpl.scala:750) ``` ### Does this PR introduce _any_ user-facing change? Yes, bug fix. ### How was this patch tested? Add test. Closes #31632 from ulysses-you/SPARK-34515. Authored-by: ulysses-you Signed-off-by: Wenchen Fan (cherry picked from commit 999d3b89b6df14a5ccb94ffc2ffadb82964e9f7d) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/hive/client/HiveShim.scala | 4 +++- .../org/apache/spark/sql/hive/client/FiltersSuite.scala | 8 ++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index ed088648bc20a..8ccb17ce35925 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -769,7 +769,9 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { case InSet(child, values) if useAdvanced && values.size > inSetThreshold => val dataType = child.dataType - val sortedValues = values.toSeq.sorted(TypeUtils.getInterpretedOrdering(dataType)) + // Skip null here is safe, more details could see at ExtractableLiterals. + val sortedValues = values.filter(_ != null).toSeq + .sorted(TypeUtils.getInterpretedOrdering(dataType)) convert(And(GreaterThanOrEqual(child, Literal(sortedValues.head, dataType)), LessThanOrEqual(child, Literal(sortedValues.last, dataType)))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 12ed0e5305299..6962f9dd6b186 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -179,5 +179,13 @@ class FiltersSuite extends SparkFunSuite with Logging with PlanTest { } } + test("SPARK-34515: Fix NPE if InSet contains null value during getPartitionsByFilter") { + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "2") { + val filter = InSet(a("p", IntegerType), Set(null, 1, 2)) + val converted = shim.convertFilters(testTable, Seq(filter), conf.sessionLocalTimeZone) + assert(converted == "(p >= 1 and p <= 2)") + } + } + private def a(name: String, dataType: DataType) = AttributeReference(name, dataType)() } From eee08d71fbf76c661f992b52c43917bc61564739 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 25 Feb 2021 18:07:39 +0800 Subject: [PATCH 017/169] [SPARK-34436][SQL] DPP support LIKE ANY/ALL expression ### What changes were proposed in this pull request? This pr make DPP support LIKE ANY/ALL expression: ```sql SELECT date_id, product_id FROM fact_sk f JOIN dim_store s ON f.store_id = s.store_id WHERE s.country LIKE ANY ('%D%E%', '%A%B%') ``` ### Why are the changes needed? Improve query performance. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. Closes #31563 from wangyum/SPARK-34436. Lead-authored-by: Yuming Wang Co-authored-by: Yuming Wang Signed-off-by: Wenchen Fan (cherry picked from commit 4a3200b08ac3e7733b5a3dc7271d35e6872c5967) Signed-off-by: Wenchen Fan --- .../dynamicpruning/PartitionPruning.scala | 1 + .../sql/DynamicPartitionPruningSuite.scala | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala index e30f9b65a2c2c..7fac91a337adc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala @@ -163,6 +163,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { case _: BinaryComparison => true case _: In | _: InSet => true case _: StringPredicate => true + case _: MultiLikeBase => true case _ => false } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 55437aaa47298..db7b0dd4b67e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1361,6 +1361,26 @@ abstract class DynamicPartitionPruningSuiteBase checkAnswer(df, Nil) } } + + test("SPARK-34436: DPP support LIKE ANY/ALL expression") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + val df = sql( + """ + |SELECT date_id, product_id FROM fact_sk f + |JOIN dim_store s + |ON f.store_id = s.store_id WHERE s.country LIKE ANY ('%D%E%', '%A%B%') + """.stripMargin) + + checkPartitionPruningPredicate(df, false, true) + + checkAnswer(df, + Row(1030, 2) :: + Row(1040, 2) :: + Row(1050, 2) :: + Row(1060, 2) :: Nil + ) + } + } } class DynamicPartitionPruningSuiteAEOff extends DynamicPartitionPruningSuiteBase { From 91f2a9e86a38245dde3aa8b37ee2619907545f3f Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Fri, 26 Feb 2021 21:29:14 +0900 Subject: [PATCH 018/169] [SPARK-34550][SQL] Skip InSet null value during push filter to Hive metastore ### What changes were proposed in this pull request? Skip `InSet` null value during push filter to Hive metastore. ### Why are the changes needed? If `InSet` contains a null value, we should skip it and push other values to metastore. To keep same behavior with `In`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Add test. Closes #31659 from ulysses-you/SPARK-34550. Authored-by: ulysses-you Signed-off-by: HyukjinKwon (cherry picked from commit 82267acfe8c78a70d56a6ae6ab9a1135c0dc0836) Signed-off-by: HyukjinKwon --- .../apache/spark/sql/hive/client/HiveShim.scala | 4 ++-- .../spark/sql/hive/client/FiltersSuite.scala | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 8ccb17ce35925..db67480ceb77a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -700,7 +700,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } def unapply(values: Set[Any]): Option[Seq[String]] = { - val extractables = values.toSeq.map(valueToLiteralString.lift) + val extractables = values.filter(_ != null).toSeq.map(valueToLiteralString.lift) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { @@ -715,7 +715,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } def unapply(values: Set[Any]): Option[Seq[String]] = { - val extractables = values.toSeq.map(valueToLiteralString.lift) + val extractables = values.filter(_ != null).toSeq.map(valueToLiteralString.lift) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 6962f9dd6b186..79b34bd141de3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -187,5 +187,20 @@ class FiltersSuite extends SparkFunSuite with Logging with PlanTest { } } + test("SPARK-34538: Skip InSet null value during push filter to Hive metastore") { + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "3") { + val intFilter = InSet(a("p", IntegerType), Set(null, 1, 2)) + val intConverted = shim.convertFilters(testTable, Seq(intFilter), conf.sessionLocalTimeZone) + assert(intConverted == "(p = 1 or p = 2)") + } + + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "3") { + val dateFilter = InSet(a("p", DateType), Set(null, + Literal(Date.valueOf("2020-01-01")).eval(), Literal(Date.valueOf("2021-01-01")).eval())) + val dateConverted = shim.convertFilters(testTable, Seq(dateFilter), conf.sessionLocalTimeZone) + assert(dateConverted == "(p = 2020-01-01 or p = 2021-01-01)") + } + } + private def a(name: String, dataType: DataType) = AttributeReference(name, dataType)() } From d78ae65f01094f3e2dd8a2b14b7da3c7ff5905d4 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 1 Mar 2021 13:55:35 +0900 Subject: [PATCH 019/169] [SPARK-34556][SQL] Checking duplicate static partition columns should respect case sensitive conf ### What changes were proposed in this pull request? This PR makes partition spec parsing respect case sensitive conf. ### Why are the changes needed? When parsing the partition spec, Spark will call `org.apache.spark.sql.catalyst.parser.ParserUtils.checkDuplicateKeys` to check if there are duplicate partition column names in the list. But this method is always case sensitive and doesn't detect duplicate partition column names when using different cases. ### Does this PR introduce _any_ user-facing change? Yep. This prevents users from writing incorrect queries such as `INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)` when they don't enable case sensitive conf. ### How was this patch tested? The new added test will fail without this change. Closes #31669 from zsxwing/SPARK-34556. Authored-by: Shixiong Zhu Signed-off-by: HyukjinKwon (cherry picked from commit 62737e140c7b04805726a33c392c297335db7b45) Signed-off-by: HyukjinKwon --- .../sql/catalyst/parser/AstBuilder.scala | 6 ++++- .../apache/spark/sql/SQLInsertTestSuite.scala | 22 +++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c3d593c47f74e..63906e11cebc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -491,7 +491,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg // Before calling `toMap`, we check duplicated keys to avoid silently ignore partition values // in partition spec like PARTITION(a='1', b='2', a='3'). The real semantical check for // partition columns will be done in analyzer. - checkDuplicateKeys(parts.toSeq, ctx) + if (conf.caseSensitiveAnalysis) { + checkDuplicateKeys(parts.toSeq, ctx) + } else { + checkDuplicateKeys(parts.map(kv => kv._1.toLowerCase(Locale.ROOT) -> kv._2).toSeq, ctx) + } parts.toMap } 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 c7446c7a9f443..67c5f12dc71dd 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 @@ -208,6 +208,28 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { checkAnswer(spark.table("t"), Row("1", null)) } } + + test("SPARK-34556: " + + "checking duplicate static partition columns should respect case sensitive conf") { + withTable("t") { + sql(s"CREATE TABLE t(i STRING, c string) USING PARQUET PARTITIONED BY (c)") + val e = intercept[AnalysisException] { + sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)") + } + assert(e.getMessage.contains("Found duplicate keys 'c'")) + } + // The following code is skipped for Hive because columns stored in Hive Metastore is always + // case insensitive and we cannot create such table in Hive Metastore. + if (!format.startsWith("hive")) { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTable("t") { + sql(s"CREATE TABLE t(i int, c string, C string) USING PARQUET PARTITIONED BY (c, C)") + sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)") + checkAnswer(spark.table("t"), Row(1, "2", "3")) + } + } + } + } } class FileSourceSQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession { From 53997008faeb5abe73e7d9cc3274cdc40bfbee3a Mon Sep 17 00:00:00 2001 From: Amandeep Sharma Date: Tue, 2 Mar 2021 17:14:15 +0800 Subject: [PATCH 020/169] [SPARK-34417][SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot **What changes were proposed in this pull request?** This PR fixes dataframe.na.fillMap() for column having a dot in the name as mentioned in [SPARK-34417](https://issues.apache.org/jira/browse/SPARK-34417). Use resolved attributes of a column for replacing null values. **Why are the changes needed?** dataframe.na.fillMap() does not work for column having a dot in the name **Does this PR introduce any user-facing change?** None **How was this patch tested?** Added unit test for the same Closes #31545 from amandeep-sharma/master. Lead-authored-by: Amandeep Sharma Co-authored-by: Amandeep Sharma Signed-off-by: Wenchen Fan (cherry picked from commit 4bda3c0f0225817456c4e423d4c85cc6b796f0c9) Signed-off-by: Wenchen Fan --- .../spark/sql/DataFrameNaFunctions.scala | 45 ++++++++++--------- .../spark/sql/DataFrameNaFunctionsSuite.scala | 25 +++++++++++ 2 files changed, 48 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index bbf0ac1dd85e9..308bb96502b19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -395,10 +395,13 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { private def fillMap(values: Seq[(String, Any)]): DataFrame = { // Error handling - values.foreach { case (colName, replaceValue) => + val attrToValue = AttributeMap(values.map { case (colName, replaceValue) => // Check column name exists - df.resolve(colName) - + val attr = df.resolve(colName) match { + case a: Attribute => a + case _ => throw new UnsupportedOperationException( + s"Nested field ${colName} is not supported.") + } // Check data type replaceValue match { case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: jl.Boolean | _: String => @@ -406,31 +409,29 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { case _ => throw new IllegalArgumentException( s"Unsupported value type ${replaceValue.getClass.getName} ($replaceValue).") } - } - - val columnEquals = df.sparkSession.sessionState.analyzer.resolver - val projections = df.schema.fields.map { f => - values.find { case (k, _) => columnEquals(k, f.name) }.map { case (_, v) => - v match { - case v: jl.Float => fillCol[Float](f, v) - case v: jl.Double => fillCol[Double](f, v) - case v: jl.Long => fillCol[Long](f, v) - case v: jl.Integer => fillCol[Integer](f, v) - case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue()) - case v: String => fillCol[String](f, v) - } - }.getOrElse(df.col(f.name)) + attr -> replaceValue + }) + + val output = df.queryExecution.analyzed.output + val projections = output.map { + attr => attrToValue.get(attr).map { + case v: jl.Float => fillCol[Float](attr, v) + case v: jl.Double => fillCol[Double](attr, v) + case v: jl.Long => fillCol[Long](attr, v) + case v: jl.Integer => fillCol[Integer](attr, v) + case v: jl.Boolean => fillCol[Boolean](attr, v.booleanValue()) + case v: String => fillCol[String](attr, v) + }.getOrElse(Column(attr)) } df.select(projections : _*) } /** - * Returns a [[Column]] expression that replaces null value in `col` with `replacement`. - * It selects a column based on its name. + * Returns a [[Column]] expression that replaces null value in column defined by `attr` + * with `replacement`. */ - private def fillCol[T](col: StructField, replacement: T): Column = { - val quotedColName = "`" + col.name + "`" - fillCol(col.dataType, col.name, df.col(quotedColName), replacement) + private def fillCol[T](attr: Attribute, replacement: T): Column = { + fillCol(attr.dataType, attr.name, Column(attr), replacement) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 091877f7cac37..23c2349f89574 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -460,4 +460,29 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil) } + + test("SPARK-34417 - test fillMap() for column with a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col") + .na.fill(Map("`ColWith.Dot`" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } + + test("SPARK-34417 - test fillMap() for qualified-column with a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col").as("testDF") + .na.fill(Map("testDF.`ColWith.Dot`" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } + + test("SPARK-34417 - test fillMap() for column without a dot in the name" + + " and dataframe with another column having a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("Col", "ColWith.Dot") + .na.fill(Map("Col" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } } From 59bd127e953a1a04f112dce92cd503d2a98f3568 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 2 Mar 2021 17:27:13 +0800 Subject: [PATCH 021/169] [SPARK-34547][SQL] Only use metadata columns for resolution as last resort ### What changes were proposed in this pull request? Today, child expressions may be resolved based on "real" or metadata output attributes. We should prefer the real attribute during resolution if one exists. ### Why are the changes needed? Today, attempting to resolve an expression when there is a "real" output attribute and a metadata attribute with the same name results in resolution failure. This is likely unexpected, as the user may not know about the metadata attribute. ### Does this PR introduce _any_ user-facing change? Yes. Previously, the user would see an error message when resolving a column with the same name as a "real" output attribute and a metadata attribute as below: ``` org.apache.spark.sql.AnalysisException: Reference 'index' is ambiguous, could be: testcat.ns1.ns2.tableTwo.index, testcat.ns1.ns2.tableOne.index.; line 1 pos 71 at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:363) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveChildren(LogicalPlan.scala:107) ``` Now, resolution succeeds and provides the "real" output attribute. ### How was this patch tested? Added a unit test. Closes #31654 from karenfeng/fallback-resolve-metadata. Authored-by: Karen Feng Signed-off-by: Wenchen Fan (cherry picked from commit 2e54d68eb94cf39b59166f2b1bbb8f6c317760b8) Signed-off-by: Wenchen Fan --- .../catalyst/plans/logical/LogicalPlan.scala | 6 ++-- .../sql/connector/DataSourceV2SQLSuite.scala | 31 +++++++++++++++++++ 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ad5c3fd74e9b5..781e4c21c3058 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -89,8 +89,9 @@ abstract class LogicalPlan } } - private[this] lazy val childAttributes = - AttributeSeq(children.flatMap(c => c.output ++ c.metadataOutput)) + private[this] lazy val childAttributes = AttributeSeq(children.flatMap(_.output)) + + private[this] lazy val childMetadataAttributes = AttributeSeq(children.flatMap(_.metadataOutput)) private[this] lazy val outputAttributes = AttributeSeq(output) @@ -103,6 +104,7 @@ abstract class LogicalPlan nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = childAttributes.resolve(nameParts, resolver) + .orElse(childMetadataAttributes.resolve(nameParts, resolver)) /** * Optionally resolves the given strings to a [[NamedExpression]] based on the output of this 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 0e12eba84eb03..38888c3fdd80b 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 @@ -2649,6 +2649,37 @@ class DataSourceV2SQLSuite } } + test("SPARK-34547: metadata columns are resolved last") { + val t1 = s"${catalogAndNamespace}tableOne" + val t2 = "t2" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + withTempView(t2) { + sql(s"CREATE TEMPORARY VIEW $t2 AS SELECT * FROM " + + s"VALUES (1, -1), (2, -2), (3, -3) AS $t2(id, index)") + + val sqlQuery = spark.sql(s"SELECT $t1.id, $t2.id, data, index, $t1.index, $t2.index FROM " + + s"$t1 JOIN $t2 WHERE $t1.id = $t2.id") + val t1Table = spark.table(t1) + val t2Table = spark.table(t2) + val dfQuery = t1Table.join(t2Table, t1Table.col("id") === t2Table.col("id")) + .select(s"$t1.id", s"$t2.id", "data", "index", s"$t1.index", s"$t2.index") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, + Seq( + Row(1, 1, "a", -1, 0, -1), + Row(2, 2, "b", -2, 0, -2), + Row(3, 3, "c", -3, 0, -3) + ) + ) + } + } + } + } + test("SPARK-33505: insert into partitioned table") { val t = "testpart.ns1.ns2.tbl" withTable(t) { From bcf662e78054470a821abc0dde6bd9afb36b2e4a Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Wed, 3 Mar 2021 12:22:51 +0900 Subject: [PATCH 022/169] [SPARK-34596][SQL] Use Utils.getSimpleName to avoid hitting Malformed class name in NewInstance.doGenCode ### What changes were proposed in this pull request? Use `Utils.getSimpleName` to avoid hitting `Malformed class name` error in `NewInstance.doGenCode`. ### Why are the changes needed? On older JDK versions (e.g. JDK8u), nested Scala classes may trigger `java.lang.Class.getSimpleName` to throw an `java.lang.InternalError: Malformed class name` error. In this particular case, creating an `ExpressionEncoder` on such a nested Scala class would create a `NewInstance` expression under the hood, which will trigger the problem during codegen. Similar to https://github.com/apache/spark/pull/29050, we should use Spark's `Utils.getSimpleName` utility function in place of `Class.getSimpleName` to avoid hitting the issue. There are two other occurrences of `java.lang.Class.getSimpleName` in the same file, but they're safe because they're only guaranteed to be only used on Java classes, which don't have this problem, e.g.: ```scala // Make a copy of the data if it's unsafe-backed def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) = s"$value instanceof ${clazz.getSimpleName}? ${value}.copy() : $value" val genFunctionValue: String = lambdaFunction.dataType match { case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value) case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value) case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value) case _ => genFunction.value } ``` The Unsafe-* family of types are all Java types, so they're okay. ### Does this PR introduce _any_ user-facing change? Fixes a bug that throws an error when using `ExpressionEncoder` on some nested Scala types, otherwise no changes. ### How was this patch tested? Added a test case to `org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite`. It'll fail on JDK8u before the fix, and pass after the fix. Closes #31709 from rednaxelafx/spark-34596-master. Authored-by: Kris Mok Signed-off-by: HyukjinKwon (cherry picked from commit ecf4811764f1ef91954c865a864e0bf6691f99a6) Signed-off-by: HyukjinKwon --- .../sql/catalyst/expressions/objects/objects.scala | 2 +- .../catalyst/encoders/ExpressionEncoderSuite.scala | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index f391b3128cf41..8801c7d3a9271 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -489,7 +489,7 @@ case class NewInstance( // that might be defined on the companion object. case 0 => s"$className$$.MODULE$$.apply($argString)" case _ => outer.map { gen => - s"${gen.value}.new ${cls.getSimpleName}($argString)" + s"${gen.value}.new ${Utils.getSimpleName(cls)}($argString)" }.getOrElse { s"new $className($argString)" } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index f2598a925e08e..26352648638c7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -205,6 +205,18 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(Array(Option(InnerClass(1))), "array of optional inner class") + // holder class to trigger Class.getSimpleName issue + object MalformedClassObject extends Serializable { + case class MalformedNameExample(x: Int) + } + + { + OuterScopes.addOuterScope(MalformedClassObject) + encodeDecodeTest( + MalformedClassObject.MalformedNameExample(42), + "nested Scala class should work") + } + productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true)) productTest( From 0ba776739fbb6930d6dbf55c9970b91d67cc723e Mon Sep 17 00:00:00 2001 From: yuhaiyang Date: Wed, 3 Mar 2021 12:25:38 +0800 Subject: [PATCH 023/169] [SPARK-34534] Fix blockIds order when use FetchShuffleBlocks to fetch blocks ### What changes were proposed in this pull request? Fix a problems which can lead to data correctness after part blocks retry in `OneForOneBlockFetcher` when use `FetchShuffleBlocks` . ### Why are the changes needed? This is a data correctness bug, It's is no problems when use old protocol to send `OpenBlocks` before fetch chunks in `OneForOneBlockFetcher`; In latest branch, `OpenBlocks` has been replaced to `FetchShuffleBlocks`. Howerver, `FetchShuffleBlocks` read shuffle blocks order is not the same as `blockIds` in `OneForOneBlockFetcher`; the `blockIds` is used to match blockId with shuffle data with index, now it is out of order; It will lead to read wrong block chunk when some blocks fetch failed in `OneForOneBlockFetcher`, it will retry the rest of the blocks in `blockIds` based on the `blockIds`'s order. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Closes #31643 from seayoun/yuhaiyang_fix_use_FetchShuffleBlocks_order. Lead-authored-by: yuhaiyang Co-authored-by: yuhaiyang Signed-off-by: Wenchen Fan (cherry picked from commit 4e438196114eff2e1fc4dd726fdc1bda1af267da) Signed-off-by: Wenchen Fan --- .../shuffle/OneForOneBlockFetcher.java | 53 ++++++++++++++----- .../shuffle/OneForOneBlockFetcherSuite.java | 42 +++++++++++++++ 2 files changed, 81 insertions(+), 14 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index ec2e3dce661d9..0b7eaa6225a41 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -21,7 +21,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; +import java.util.LinkedHashMap; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; @@ -81,7 +81,6 @@ public OneForOneBlockFetcher( TransportConf transportConf, DownloadFileManager downloadFileManager) { this.client = client; - this.blockIds = blockIds; this.listener = listener; this.chunkCallback = new ChunkCallback(); this.transportConf = transportConf; @@ -90,8 +89,10 @@ public OneForOneBlockFetcher( throw new IllegalArgumentException("Zero-sized blockIds array"); } if (!transportConf.useOldFetchProtocol() && isShuffleBlocks(blockIds)) { - this.message = createFetchShuffleBlocksMsg(appId, execId, blockIds); + this.blockIds = new String[blockIds.length]; + this.message = createFetchShuffleBlocksMsgAndBuildBlockIds(appId, execId, blockIds); } else { + this.blockIds = blockIds; this.message = new OpenBlocks(appId, execId, blockIds); } } @@ -106,17 +107,16 @@ private boolean isShuffleBlocks(String[] blockIds) { } /** - * Analyze the pass in blockIds and create FetchShuffleBlocks message. - * The blockIds has been sorted by mapId and reduceId. It's produced in - * org.apache.spark.MapOutputTracker.convertMapStatuses. + * Create FetchShuffleBlocks message and rebuild internal blockIds by + * analyzing the pass in blockIds. */ - private FetchShuffleBlocks createFetchShuffleBlocksMsg( + private FetchShuffleBlocks createFetchShuffleBlocksMsgAndBuildBlockIds( String appId, String execId, String[] blockIds) { String[] firstBlock = splitBlockId(blockIds[0]); int shuffleId = Integer.parseInt(firstBlock[1]); boolean batchFetchEnabled = firstBlock.length == 5; - HashMap> mapIdToReduceIds = new HashMap<>(); + LinkedHashMap mapIdToBlocksInfo = new LinkedHashMap<>(); for (String blockId : blockIds) { String[] blockIdParts = splitBlockId(blockId); if (Integer.parseInt(blockIdParts[1]) != shuffleId) { @@ -124,23 +124,36 @@ private FetchShuffleBlocks createFetchShuffleBlocksMsg( ", got:" + blockId); } long mapId = Long.parseLong(blockIdParts[2]); - if (!mapIdToReduceIds.containsKey(mapId)) { - mapIdToReduceIds.put(mapId, new ArrayList<>()); + if (!mapIdToBlocksInfo.containsKey(mapId)) { + mapIdToBlocksInfo.put(mapId, new BlocksInfo()); } - mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[3])); + BlocksInfo blocksInfoByMapId = mapIdToBlocksInfo.get(mapId); + blocksInfoByMapId.blockIds.add(blockId); + blocksInfoByMapId.reduceIds.add(Integer.parseInt(blockIdParts[3])); if (batchFetchEnabled) { // When we read continuous shuffle blocks in batch, we will reuse reduceIds in // FetchShuffleBlocks to store the start and end reduce id for range // [startReduceId, endReduceId). assert(blockIdParts.length == 5); - mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[4])); + blocksInfoByMapId.reduceIds.add(Integer.parseInt(blockIdParts[4])); } } - long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); + long[] mapIds = Longs.toArray(mapIdToBlocksInfo.keySet()); int[][] reduceIdArr = new int[mapIds.length][]; + int blockIdIndex = 0; for (int i = 0; i < mapIds.length; i++) { - reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); + BlocksInfo blocksInfoByMapId = mapIdToBlocksInfo.get(mapIds[i]); + reduceIdArr[i] = Ints.toArray(blocksInfoByMapId.reduceIds); + + // The `blockIds`'s order must be same with the read order specified in in FetchShuffleBlocks + // because the shuffle data's return order should match the `blockIds`'s order to ensure + // blockId and data match. + for (int j = 0; j < blocksInfoByMapId.blockIds.size(); j++) { + this.blockIds[blockIdIndex++] = blocksInfoByMapId.blockIds.get(j); + } } + assert(blockIdIndex == this.blockIds.length); + return new FetchShuffleBlocks( appId, execId, shuffleId, mapIds, reduceIdArr, batchFetchEnabled); } @@ -157,6 +170,18 @@ private String[] splitBlockId(String blockId) { return blockIdParts; } + /** The reduceIds and blocks in a single mapId */ + private class BlocksInfo { + + final ArrayList reduceIds; + final ArrayList blockIds; + + BlocksInfo() { + this.reduceIds = new ArrayList<>(); + this.blockIds = new ArrayList<>(); + } + } + /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ private class ChunkCallback implements ChunkReceivedCallback { @Override diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 285eedb39c65c..a7eb59d366966 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -201,6 +201,48 @@ public void testEmptyBlockFetch() { } } + @Test + public void testFetchShuffleBlocksOrder() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + blocks.put("shuffle_0_2_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[2]))); + blocks.put("shuffle_0_10_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, + new long[]{0, 2, 10}, new int[][]{{0}, {1}, {2}}, false), + conf); + + for (int chunkIndex = 0; chunkIndex < blockIds.length; chunkIndex++) { + String blockId = blockIds[chunkIndex]; + verify(listener).onBlockFetchSuccess(blockId, blocks.get(blockId)); + } + } + + @Test + public void testBatchFetchShuffleBlocksOrder() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_1_2", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + blocks.put("shuffle_0_2_2_3", new NioManagedBuffer(ByteBuffer.wrap(new byte[2]))); + blocks.put("shuffle_0_10_3_4", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, + new long[]{0, 2, 10}, new int[][]{{1, 2}, {2, 3}, {3, 4}}, true), + conf); + + for (int chunkIndex = 0; chunkIndex < blockIds.length; chunkIndex++) { + String blockId = blockIds[chunkIndex]; + verify(listener).onBlockFetchSuccess(blockId, blocks.get(blockId)); + } + } + /** * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which * simply returns the given (BlockId, Block) pairs. From b3500ce4f0e089eddedfae1fb382a37a40d60280 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 3 Mar 2021 22:07:41 +0800 Subject: [PATCH 024/169] [SPARK-34555][SQL] Resolve metadata output from DataFrame Add metadataOutput as a fallback to resolution. Builds off https://github.com/apache/spark/pull/31654. The metadata columns could not be resolved via `df.col("metadataColName")` from the DataFrame API. Yes, the metadata columns can now be resolved as described above. Scala unit test. Closes #31668 from karenfeng/spark-34555. Authored-by: Karen Feng Signed-off-by: Wenchen Fan (cherry picked from commit b01dd12805f7b40318f183ee48bc0012bb4e847f) Signed-off-by: Wenchen Fan --- .../catalyst/plans/logical/LogicalPlan.scala | 5 +++- .../sql/connector/DataSourceV2SQLSuite.scala | 23 ++++++++++++++++++- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 781e4c21c3058..bdf37d040eb79 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -95,6 +95,8 @@ abstract class LogicalPlan private[this] lazy val outputAttributes = AttributeSeq(output) + private[this] lazy val outputMetadataAttributes = AttributeSeq(metadataOutput) + /** * Optionally resolves the given strings to a [[NamedExpression]] using the input from all child * nodes of this LogicalPlan. The attribute is expressed as @@ -115,6 +117,7 @@ abstract class LogicalPlan nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = outputAttributes.resolve(nameParts, resolver) + .orElse(outputMetadataAttributes.resolve(nameParts, resolver)) /** * Given an attribute name, split it to name parts by dot, but @@ -124,7 +127,7 @@ abstract class LogicalPlan def resolveQuoted( name: String, resolver: Resolver): Option[NamedExpression] = { - outputAttributes.resolve(UnresolvedAttribute.parseAttributeName(name), resolver) + resolve(UnresolvedAttribute.parseAttributeName(name), resolver) } /** 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 38888c3fdd80b..3e066c977c15a 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 @@ -35,7 +35,7 @@ import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.SimpleScanSource -import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{BooleanType, LongType, MetadataBuilder, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.types.UTF8String @@ -2699,6 +2699,27 @@ class DataSourceV2SQLSuite } } + test("SPARK-34555: Resolve DataFrame metadata column") { + val tbl = s"${catalogAndNamespace}table" + withTable(tbl) { + sql(s"CREATE TABLE $tbl (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $tbl VALUES (1, 'a'), (2, 'b'), (3, 'c')") + val table = spark.table(tbl) + val dfQuery = table.select( + table.col("id"), + table.col("data"), + table.col("index"), + table.col("_partition") + ) + + checkAnswer( + dfQuery, + Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3")) + ) + } + } + private def testNotSupportedV2Command( sqlCommand: String, sqlParams: String, From c41b543773cc0c9b1199652fc66d84b797d68c35 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 4 Mar 2021 11:29:34 +0900 Subject: [PATCH 025/169] [SPARK-34584][SQL] Static partition should also follow StoreAssignmentPolicy when insert into v2 tables ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/27597 and simply apply the fix in the v2 table insertion code path. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? yes, now v2 table insertion with static partitions also follow StoreAssignmentPolicy. ### How was this patch tested? moved the test from https://github.com/apache/spark/pull/27597 to the general test suite `SQLInsertTestSuite`, which covers DS v2, file source, and hive tables. Closes #31726 from cloud-fan/insert. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon (cherry picked from commit 8f1eec4d138da604b890111d0a6daaef86d44ef2) Signed-off-by: HyukjinKwon --- .../sql/catalyst/analysis/Analyzer.scala | 6 +++- .../apache/spark/sql/SQLInsertTestSuite.scala | 29 ++++++++++++++++++- .../spark/sql/sources/InsertSuite.scala | 21 -------------- 3 files changed, 33 insertions(+), 23 deletions(-) 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 bf8003105ada6..f71139fafcc04 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 @@ -1304,7 +1304,11 @@ class Analyzer(override val catalogManager: CatalogManager) relation.output.flatMap { col => outputNameToStaticName.get(col.name).flatMap(staticPartitions.get) match { case Some(staticValue) => - Some(Alias(Cast(Literal(staticValue), col.dataType), col.name)()) + // SPARK-30844: try our best to follow StoreAssignmentPolicy for static partition + // values but not completely follow because we can't do static type checking due to + // the reason that the parser has erased the type info of static partition values + // and converted them to string. + Some(Alias(AnsiCast(Literal(staticValue), col.dataType), col.name)()) case _ if queryColumns.hasNext => Some(queryColumns.next) case _ => 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 67c5f12dc71dd..a0943437bc8b9 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 @@ -23,7 +23,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} /** - * The base trait for DML - insert syntax + * The base trait for SQL INSERT. */ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { @@ -230,6 +230,33 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { } } } + + test("SPARK-30844: static partition should also follow StoreAssignmentPolicy") { + val testingPolicies = if (format == "foo") { + // DS v2 doesn't support the legacy policy + Seq(SQLConf.StoreAssignmentPolicy.ANSI, SQLConf.StoreAssignmentPolicy.STRICT) + } else { + SQLConf.StoreAssignmentPolicy.values + } + testingPolicies.foreach { policy => + withSQLConf( + SQLConf.STORE_ASSIGNMENT_POLICY.key -> policy.toString) { + withTable("t") { + sql("create table t(a int, b string) using parquet partitioned by (a)") + policy match { + case SQLConf.StoreAssignmentPolicy.ANSI | SQLConf.StoreAssignmentPolicy.STRICT => + val errorMsg = intercept[NumberFormatException] { + sql("insert into t partition(a='ansi') values('ansi')") + }.getMessage + assert(errorMsg.contains("invalid input syntax for type numeric: ansi")) + case SQLConf.StoreAssignmentPolicy.LEGACY => + sql("insert into t partition(a='ansi') values('ansi')") + checkAnswer(sql("select * from t"), Row("ansi", null) :: Nil) + } + } + } + } + } } class FileSourceSQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession { 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 aaf8765c04425..bce55ac34419f 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 @@ -797,27 +797,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } - test("SPARK-30844: static partition should also follow StoreAssignmentPolicy") { - SQLConf.StoreAssignmentPolicy.values.foreach { policy => - withSQLConf( - SQLConf.STORE_ASSIGNMENT_POLICY.key -> policy.toString) { - withTable("t") { - sql("create table t(a int, b string) using parquet partitioned by (a)") - policy match { - case SQLConf.StoreAssignmentPolicy.ANSI | SQLConf.StoreAssignmentPolicy.STRICT => - val errorMsg = intercept[NumberFormatException] { - sql("insert into t partition(a='ansi') values('ansi')") - }.getMessage - assert(errorMsg.contains("invalid input syntax for type numeric: ansi")) - case SQLConf.StoreAssignmentPolicy.LEGACY => - sql("insert into t partition(a='ansi') values('ansi')") - checkAnswer(sql("select * from t"), Row("ansi", null) :: Nil) - } - } - } - } - } - test("SPARK-24860: dynamic partition overwrite specified per source without catalog table") { withTempPath { path => Seq((1, 1), (2, 2)).toDF("i", "part") From 463e130d855b95254bb40422c4cf4da8ff2cc766 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 4 Mar 2021 15:12:53 +0800 Subject: [PATCH 026/169] [SPARK-34599][SQL] Fix the issue that INSERT INTO OVERWRITE doesn't support partition columns containing dot for DSv2 ### What changes were proposed in this pull request? `ResolveInsertInto.staticDeleteExpression` should use `UnresolvedAttribute.quoted` to create the delete expression so that we will treat the entire `attr.name` as a column name. ### Why are the changes needed? When users use `dot` in a partition column name, queries like ```INSERT OVERWRITE $t1 PARTITION (`a.b` = 'a') (`c.d`) VALUES('b')``` is not working. ### Does this PR introduce _any_ user-facing change? Without this test, the above query will throw ``` [info] org.apache.spark.sql.AnalysisException: cannot resolve '`a.b`' given input columns: [a.b, c.d]; [info] 'OverwriteByExpression RelationV2[a.b#17, c.d#18] default.tbl, ('a.b <=> cast(a as string)), false [info] +- Project [a.b#19, ansi_cast(col1#16 as string) AS c.d#20] [info] +- Project [cast(a as string) AS a.b#19, col1#16] [info] +- LocalRelation [col1#16] ``` With the fix, the query will run correctly. ### How was this patch tested? The new added test. Closes #31713 from zsxwing/SPARK-34599. Authored-by: Shixiong Zhu Signed-off-by: Wenchen Fan (cherry picked from commit 53e4dba7c489ac5c0ad61f0121c4e247de5b485c) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 +++- .../apache/spark/sql/connector/InsertIntoTests.scala | 10 ++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) 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 f71139fafcc04..771b817955854 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 @@ -1336,7 +1336,9 @@ class Analyzer(override val catalogManager: CatalogManager) // ResolveOutputRelation runs, using the query's column names that will match the // table names at that point. because resolution happens after a future rule, create // an UnresolvedAttribute. - EqualNullSafe(UnresolvedAttribute(attr.name), Cast(Literal(value), attr.dataType)) + EqualNullSafe( + UnresolvedAttribute.quoted(attr.name), + Cast(Literal(value), attr.dataType)) case None => throw QueryCompilationErrors.unknownStaticPartitionColError(name) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index 2cc7a1f994645..ad730376b2e3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -477,5 +477,15 @@ trait InsertIntoSQLOnlyTests verifyTable(t1, spark.table(view)) } } + + test("SPARK-34599: InsertInto: overwrite - dot in the partition column name - static mode") { + import testImplicits._ + val t1 = "tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (`a.b` string, `c.d` string) USING $v2Format PARTITIONED BY (`a.b`)") + sql(s"INSERT OVERWRITE $t1 PARTITION (`a.b` = 'a') (`c.d`) VALUES('b')") + verifyTable(t1, Seq("a" -> "b").toDF("id", "data")) + } + } } } From f3fdc8d68f7b7ff7e2d76db374f69c49219c3a67 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Thu, 4 Mar 2021 20:42:47 +0800 Subject: [PATCH 027/169] [SPARK-34567][SQL] CreateTableAsSelect should update metrics too ### What changes were proposed in this pull request? For command `CreateTableAsSelect` we use `InsertIntoHiveTable`, `InsertIntoHadoopFsRelationCommand` to insert data. We will update metrics of `InsertIntoHiveTable`, `InsertIntoHadoopFsRelationCommand` in `FileFormatWriter.write()`, but we only show CreateTableAsSelectCommand in WebUI SQL Tab. We need to update `CreateTableAsSelectCommand`'s metrics too. Before this PR: ![image](https://user-images.githubusercontent.com/46485123/109411226-81f44480-79db-11eb-99cb-b9686b15bf61.png) After this PR: ![image](https://user-images.githubusercontent.com/46485123/109411232-8ae51600-79db-11eb-9111-3bea0bc2d475.png) ![image](https://user-images.githubusercontent.com/46485123/109905192-62aa2f80-7cd9-11eb-91f9-04b16c9238ae.png) ### Why are the changes needed? Complete SQL Metrics ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested?