From ab49dc21e43822abef5067f959e474c4c8dcfdff Mon Sep 17 00:00:00 2001 From: Tengfei Huang Date: Fri, 30 Sep 2022 09:04:39 +0800 Subject: [PATCH 1/8] [SPARK-39853][CORE] Support stage level task resource profile for standalone cluster when dynamic allocation disabled ### What changes were proposed in this pull request? Currently stage level scheduling works for yarn/k8s/standalone cluster when dynamic allocation is enabled, and spark app will acquire executors with different resource profiles and assign tasks to executors with the same resource profile id. This PR proposed to add stage level scheduling when dynamic allocation is off. In this case, spark app will only have executors with default resource profiles, but different `Stages` can still customize their task resource requests which should be compatible with default resource profile executor resources. And all these `Stages` with different task resource requests will reuse/share the same set of executors with default resource profile. And this PR proposed to: 1. Introduces a new special `ResourceProfile`: `TaskResourceProfile`, it can be used to describe different task resource requests when dynamic allocation is off. And tasks bind to this `TaskResourceProfile` will reuse executors with default resource profile. `Exception` should be thrown if executors with default resource profile can not fulfill the task resource requests. ``` class TaskResourceProfile(override val taskResources: Map[String, TaskResourceRequest]) extends ResourceProfile( ResourceProfile.getOrCreateDefaultProfile(SparkEnv.get.conf).executorResources, taskResources) ``` 2. `DADScheduler` and `TaskScheduler` will schedule tasks with customized `ResourceProfile` based on resource profile type and resource profile Id, taskSets with `TaskResourceProfile` can be scheduled to executors with `DEFAULT_RESOURCE_PROFILE_ID` and other taskSets can be scheduled to executors with exactly same resource profile id. ### Why are the changes needed? When dynamic allocation is disabled, we can also leverage stage level schedule to customize task resource requests for different stages. ### Does this PR introduce _any_ user-facing change? Spark users can specify `TaskResourceProfile` to customize task resource requests for different stages when dynamic allocation is off. ### How was this patch tested? New UTs added. Closes #37268 from ivoson/stage-schedule-dynamic-off. Lead-authored-by: Tengfei Huang Co-authored-by: Huang Tengfei Signed-off-by: Yi Wu --- .../spark/deploy/ApplicationDescription.scala | 3 +- .../spark/deploy/master/ApplicationInfo.scala | 4 +- .../spark/resource/ResourceProfile.scala | 63 +++++++++--- .../resource/ResourceProfileBuilder.scala | 6 +- .../resource/ResourceProfileManager.scala | 70 +++++++++---- .../apache/spark/resource/ResourceUtils.scala | 6 +- .../apache/spark/scheduler/DAGScheduler.scala | 9 +- .../spark/scheduler/TaskSchedulerImpl.scala | 19 ++-- .../ResourceProfileManagerSuite.scala | 37 +++++++ .../spark/resource/ResourceProfileSuite.scala | 65 ++++++++++++- .../spark/scheduler/DAGSchedulerSuite.scala | 19 +++- .../scheduler/TaskSchedulerImplSuite.scala | 97 ++++++++++++++++++- docs/configuration.md | 4 +- docs/spark-standalone.md | 4 +- 14 files changed, 344 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 39c2af0184643..67d0d851b60fa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -20,7 +20,6 @@ package org.apache.spark.deploy import java.net.URI import org.apache.spark.resource.{ResourceProfile, ResourceRequirement, ResourceUtils} -import org.apache.spark.resource.ResourceProfile.getCustomExecutorResources private[spark] case class ApplicationDescription( name: String, @@ -40,7 +39,7 @@ private[spark] case class ApplicationDescription( def coresPerExecutor: Option[Int] = defaultProfile.getExecutorCores def resourceReqsPerExecutor: Seq[ResourceRequirement] = ResourceUtils.executorResourceRequestToRequirement( - getCustomExecutorResources(defaultProfile).values.toSeq.sortBy(_.resourceName)) + defaultProfile.getCustomExecutorResources().values.toSeq.sortBy(_.resourceName)) override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index a2926ca64bcc3..e66933b84af55 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.resource.{ResourceInformation, ResourceProfile, ResourceUtils} -import org.apache.spark.resource.ResourceProfile.{getCustomExecutorResources, DEFAULT_RESOURCE_PROFILE_ID} +import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -101,7 +101,7 @@ private[spark] class ApplicationInfo( .map(_.toInt) .getOrElse(defaultMemoryMbPerExecutor) val customResources = ResourceUtils.executorResourceRequestToRequirement( - getCustomExecutorResources(resourceProfile).values.toSeq.sortBy(_.resourceName)) + resourceProfile.getCustomExecutorResources().values.toSeq.sortBy(_.resourceName)) rpIdToResourceDesc(resourceProfile.id) = ExecutorResourceDescription(coresPerExecutor, memoryMbPerExecutor, customResources) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 5e02c61459d1d..afd612433a7e1 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -24,7 +24,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException} import org.apache.spark.annotation.{Evolving, Since} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -94,6 +94,15 @@ class ResourceProfile( executorResources.get(ResourceProfile.MEMORY).map(_.amount) } + private[spark] def getCustomTaskResources(): Map[String, TaskResourceRequest] = { + taskResources.filterKeys(k => !k.equals(ResourceProfile.CPUS)).toMap + } + + protected[spark] def getCustomExecutorResources(): Map[String, ExecutorResourceRequest] = { + executorResources. + filterKeys(k => !ResourceProfile.allSupportedExecutorResources.contains(k)).toMap + } + /* * This function takes into account fractional amounts for the task resource requirement. * Spark only supports fractional amounts < 1 to basically allow for multiple tasks @@ -182,8 +191,8 @@ class ResourceProfile( val numPartsPerResourceMap = new mutable.HashMap[String, Int] numPartsPerResourceMap(ResourceProfile.CORES) = 1 val taskResourcesToCheck = new mutable.HashMap[String, TaskResourceRequest] - taskResourcesToCheck ++= ResourceProfile.getCustomTaskResources(this) - val execResourceToCheck = ResourceProfile.getCustomExecutorResources(this) + taskResourcesToCheck ++= this.getCustomTaskResources() + val execResourceToCheck = this.getCustomExecutorResources() execResourceToCheck.foreach { case (rName, execReq) => val taskReq = taskResources.get(rName).map(_.amount).getOrElse(0.0) numPartsPerResourceMap(rName) = 1 @@ -242,7 +251,8 @@ class ResourceProfile( // check that the task resources and executor resources are equal, but id's could be different private[spark] def resourcesEqual(rp: ResourceProfile): Boolean = { - rp.taskResources == taskResources && rp.executorResources == executorResources + rp.taskResources == taskResources && rp.executorResources == executorResources && + rp.getClass == this.getClass } override def hashCode(): Int = Seq(taskResources, executorResources).hashCode() @@ -253,6 +263,40 @@ class ResourceProfile( } } +/** + * Resource profile which only contains task resources, can be used for stage level task schedule + * when dynamic allocation is disabled, tasks will be scheduled to executors with default resource + * profile based on task resources described by this task resource profile. + * And when dynamic allocation is enabled, will require new executors for this profile based on + * the default executor resources requested at startup and assign tasks only on executors created + * with this resource profile. + * + * @param taskResources Resource requests for tasks. Mapped from the resource + * name (e.g., cores, memory, CPU) to its specific request. + */ +@Evolving +@Since("3.4.0") +private[spark] class TaskResourceProfile( + override val taskResources: Map[String, TaskResourceRequest]) + extends ResourceProfile(Map.empty, taskResources) { + + override protected[spark] def getCustomExecutorResources() + : Map[String, ExecutorResourceRequest] = { + if (SparkEnv.get == null) { + // This will be called in standalone master when dynamic allocation enabled. + return super.getCustomExecutorResources() + } + + val sparkConf = SparkEnv.get.conf + if (!Utils.isDynamicAllocationEnabled(sparkConf)) { + ResourceProfile.getOrCreateDefaultProfile(sparkConf) + .getCustomExecutorResources() + } else { + super.getCustomExecutorResources() + } + } +} + object ResourceProfile extends Logging { // task resources /** @@ -393,17 +437,6 @@ object ResourceProfile extends Logging { } } - private[spark] def getCustomTaskResources( - rp: ResourceProfile): Map[String, TaskResourceRequest] = { - rp.taskResources.filterKeys(k => !k.equals(ResourceProfile.CPUS)).toMap - } - - private[spark] def getCustomExecutorResources( - rp: ResourceProfile): Map[String, ExecutorResourceRequest] = { - rp.executorResources. - filterKeys(k => !ResourceProfile.allSupportedExecutorResources.contains(k)).toMap - } - /* * Get the number of cpus per task if its set in the profile, otherwise return the * cpus per task for the default profile. diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala index f6b30d327375e..584ff32b4475a 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala @@ -93,7 +93,11 @@ class ResourceProfileBuilder() { } def build(): ResourceProfile = { - new ResourceProfile(executorResources, taskResources) + if (_executorResources.isEmpty) { + new TaskResourceProfile(taskResources) + } else { + new ResourceProfile(executorResources, taskResources) + } } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index 489d9c3e85817..3f48aaded5c2e 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -59,35 +59,67 @@ private[spark] class ResourceProfileManager(sparkConf: SparkConf, private val testExceptionThrown = sparkConf.get(RESOURCE_PROFILE_MANAGER_TESTING) /** - * If we use anything except the default profile, it's only supported on YARN and Kubernetes - * with dynamic allocation enabled. Throw an exception if not supported. + * If we use anything except the default profile, it's supported on YARN, Kubernetes and + * Standalone with dynamic allocation enabled, and task resource profile with dynamic allocation + * disabled on Standalone. Throw an exception if not supported. */ private[spark] def isSupported(rp: ResourceProfile): Boolean = { - val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID - val notYarnOrK8sOrStandaloneAndNotDefaultProfile = - isNotDefaultProfile && !(isYarn || isK8s || isStandalone) - val YarnOrK8sOrStandaloneNotDynAllocAndNotDefaultProfile = - isNotDefaultProfile && (isYarn || isK8s || isStandalone) && !dynamicEnabled - // We want the exception to be thrown only when we are specifically testing for the - // exception or in a real application. Otherwise in all other testing scenarios we want - // to skip throwing the exception so that we can test in other modes to make testing easier. - if ((notRunningUnitTests || testExceptionThrown) && + if (rp.isInstanceOf[TaskResourceProfile] && !dynamicEnabled) { + if ((notRunningUnitTests || testExceptionThrown) && !isStandalone) { + throw new SparkException("TaskResourceProfiles are only supported for Standalone " + + "cluster for now when dynamic allocation is disabled.") + } + } else { + val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + val notYarnOrK8sOrStandaloneAndNotDefaultProfile = + isNotDefaultProfile && !(isYarn || isK8s || isStandalone) + val YarnOrK8sOrStandaloneNotDynAllocAndNotDefaultProfile = + isNotDefaultProfile && (isYarn || isK8s || isStandalone) && !dynamicEnabled + + // We want the exception to be thrown only when we are specifically testing for the + // exception or in a real application. Otherwise in all other testing scenarios we want + // to skip throwing the exception so that we can test in other modes to make testing easier. + if ((notRunningUnitTests || testExceptionThrown) && (notYarnOrK8sOrStandaloneAndNotDefaultProfile || YarnOrK8sOrStandaloneNotDynAllocAndNotDefaultProfile)) { - throw new SparkException("ResourceProfiles are only supported on YARN and Kubernetes " + - "and Standalone with dynamic allocation enabled.") - } + throw new SparkException("ResourceProfiles are only supported on YARN and Kubernetes " + + "and Standalone with dynamic allocation enabled.") + } - if (isStandalone && rp.getExecutorCores.isEmpty && - sparkConf.getOption(config.EXECUTOR_CORES.key).isEmpty) { - logWarning("Neither executor cores is set for resource profile, nor spark.executor.cores " + - "is explicitly set, you may get more executors allocated than expected. It's recommended " + - "to set executor cores explicitly. Please check SPARK-30299 for more details.") + if (isStandalone && dynamicEnabled && rp.getExecutorCores.isEmpty && + sparkConf.getOption(config.EXECUTOR_CORES.key).isEmpty) { + logWarning("Neither executor cores is set for resource profile, nor spark.executor.cores " + + "is explicitly set, you may get more executors allocated than expected. " + + "It's recommended to set executor cores explicitly. " + + "Please check SPARK-30299 for more details.") + } } true } + /** + * Check whether a task with specific taskRpId can be scheduled to executors + * with executorRpId. + * + * Here are the rules: + * 1. When dynamic allocation is disabled, only [[TaskResourceProfile]] is supported, + * and tasks with [[TaskResourceProfile]] can be scheduled to executors with default + * resource profile. + * 2. For other scenarios(when dynamic allocation is enabled), tasks can be scheduled to + * executors where resource profile exactly matches. + */ + private[spark] def canBeScheduled(taskRpId: Int, executorRpId: Int): Boolean = { + assert(resourceProfileIdToResourceProfile.contains(taskRpId) && + resourceProfileIdToResourceProfile.contains(executorRpId), + "Tasks and executors must have valid resource profile id") + val taskRp = resourceProfileFromId(taskRpId) + + // When dynamic allocation disabled, tasks with TaskResourceProfile can always reuse + // all the executors with default resource profile. + taskRpId == executorRpId || (!dynamicEnabled && taskRp.isInstanceOf[TaskResourceProfile]) + } + def addResourceProfile(rp: ResourceProfile): Unit = { isSupported(rp) var putNewProfile = false diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index 58b37269be4f8..0e18ecf0e5118 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -356,7 +356,7 @@ private[spark] object ResourceUtils extends Logging { val fileAllocated = parseAllocated(resourcesFileOpt, componentName) val fileAllocResMap = fileAllocated.map(a => (a.id.resourceName, a.toResourceInformation)).toMap // only want to look at the ResourceProfile for resources not in the resources file - val execReq = ResourceProfile.getCustomExecutorResources(resourceProfile) + val execReq = resourceProfile.getCustomExecutorResources() val filteredExecreq = execReq.filterNot { case (rname, _) => fileAllocResMap.contains(rname) } val rpAllocations = filteredExecreq.map { case (rName, execRequest) => val resourceId = new ResourceID(componentName, rName) @@ -444,8 +444,8 @@ private[spark] object ResourceUtils extends Logging { maxTaskPerExec = numTasksPerExecCores } } - val taskReq = ResourceProfile.getCustomTaskResources(rp) - val execReq = ResourceProfile.getCustomExecutorResources(rp) + val taskReq = rp.getCustomTaskResources() + val execReq = rp.getCustomExecutorResources() if (limitingResource.nonEmpty && !limitingResource.equals(ResourceProfile.CPUS)) { if ((taskCpus * maxTaskPerExec) < cores) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 475afd01d00c7..86786e64cedcf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -44,7 +44,7 @@ import org.apache.spark.network.shuffle.protocol.MergeStatuses import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.{RDD, RDDCheckpointData} -import org.apache.spark.resource.ResourceProfile +import org.apache.spark.resource.{ResourceProfile, TaskResourceProfile} import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, EXECUTOR_CORES_LOCAL_PROPERTY, PYSPARK_MEMORY_LOCAL_PROPERTY} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ @@ -592,7 +592,12 @@ private[spark] class DAGScheduler( if (x.amount > v.amount) x else v).getOrElse(v) k -> larger } - new ResourceProfile(mergedExecReq, mergedTaskReq) + + if (mergedExecReq.isEmpty) { + new TaskResourceProfile(mergedTaskReq) + } else { + new ResourceProfile(mergedExecReq, mergedTaskReq) + } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d3e27a94e2944..a6735f380f18e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -388,9 +388,10 @@ private[spark] class TaskSchedulerImpl( val execId = shuffledOffers(i).executorId val host = shuffledOffers(i).host val taskSetRpID = taskSet.taskSet.resourceProfileId - // make the resource profile id a hard requirement for now - ie only put tasksets - // on executors where resource profile exactly matches. - if (taskSetRpID == shuffledOffers(i).resourceProfileId) { + + // check whether the task can be scheduled to the executor base on resource profile. + if (sc.resourceProfileManager + .canBeScheduled(taskSetRpID, shuffledOffers(i).resourceProfileId)) { val taskResAssignmentsOpt = resourcesMeetTaskRequirements(taskSet, availableCpus(i), availableResources(i)) taskResAssignmentsOpt.foreach { taskResAssignments => @@ -463,7 +464,7 @@ private[spark] class TaskSchedulerImpl( // check if the ResourceProfile has cpus first since that is common case if (availCpus < taskCpus) return None // only look at the resource other then cpus - val tsResources = ResourceProfile.getCustomTaskResources(taskSetProf) + val tsResources = taskSetProf.getCustomTaskResources() if (tsResources.isEmpty) return Some(Map.empty) val localTaskReqAssign = HashMap[String, ResourceInformation]() // we go through all resources here so that we can make sure they match and also get what the @@ -1222,13 +1223,13 @@ private[spark] object TaskSchedulerImpl { /** * Calculate the max available task slots given the `availableCpus` and `availableResources` - * from a collection of ResourceProfiles. And only those ResourceProfiles who has the - * same id with the `rpId` can be used to calculate the task slots. + * from a collection of ResourceProfiles. And only those ResourceProfiles who can be assigned + * tasks with the `rpId` can be used to calculate the task slots. * * @param scheduler the TaskSchedulerImpl instance * @param conf SparkConf used to calculate the limiting resource and get the cpu amount per task - * @param rpId the target ResourceProfile id. Only those ResourceProfiles who has the same id - * with it can be used to calculate the task slots. + * @param rpId the ResourceProfile id for the task set. Only those ResourceProfiles who can be + * assigned with the tasks can be used to calculate the task slots. * @param availableRPIds an Array of ids of the available ResourceProfiles from the executors. * @param availableCpus an Array of the amount of available cpus from the executors. * @param availableResources an Array of the resources map from the executors. In the resource @@ -1257,7 +1258,7 @@ private[spark] object TaskSchedulerImpl { val taskLimit = resourceProfile.taskResources.get(limitingResource).map(_.amount).get availableCpus.zip(availableResources).zip(availableRPIds) - .filter { case (_, id) => id == rpId } + .filter { case (_, id) => scheduler.sc.resourceProfileManager.canBeScheduled(rpId, id) } .map { case ((cpu, resources), _) => val numTasksPerExecCores = cpu / cpusPerTask if (limitedByCpu) { diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala index aa0081356097f..e97d5c7883aa8 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -116,6 +116,43 @@ class ResourceProfileManagerSuite extends SparkFunSuite { assert(rpmanager.isSupported(immrprof)) } + test("isSupported task resource profiles with dynamic allocation disabled") { + val conf = new SparkConf().setMaster("spark://foo").set(EXECUTOR_CORES, 4) + conf.set(DYN_ALLOCATION_ENABLED, false) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + + var rpmanager = new ResourceProfileManager(conf, listenerBus) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + assert(rpmanager.isSupported(defaultProf)) + + // task resource profile. + val gpuTaskReq = new TaskResourceRequests().resource("gpu", 1) + val taskProf = new TaskResourceProfile(gpuTaskReq.requests) + assert(rpmanager.isSupported(taskProf)) + + conf.setMaster("local") + rpmanager = new ResourceProfileManager(conf, listenerBus) + val error = intercept[SparkException] { + rpmanager.isSupported(taskProf) + }.getMessage + assert(error === "TaskResourceProfiles are only supported for Standalone " + + "cluster for now when dynamic allocation is disabled.") + } + + test("isSupported task resource profiles with dynamic allocation enabled") { + val conf = new SparkConf().setMaster("spark://foo").set(EXECUTOR_CORES, 4) + conf.set(DYN_ALLOCATION_ENABLED, true) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + + val rpmanager = new ResourceProfileManager(conf, listenerBus) + + // task resource profile. + val gpuTaskReq = new TaskResourceRequests().resource("gpu", 1) + val taskProf = new TaskResourceProfile(gpuTaskReq.requests) + assert(rpmanager.isSupported(taskProf)) + } + test("isSupported with local mode") { val conf = new SparkConf().setMaster("local").set(EXECUTOR_CORES, 4) conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index 6c36f5c855547..d07b85847e77c 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.resource -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.mockito.Mockito.when +import org.scalatestplus.mockito.MockitoSugar + +import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY import org.apache.spark.resource.TestResourceIDs._ -class ResourceProfileSuite extends SparkFunSuite { +class ResourceProfileSuite extends SparkFunSuite with MockitoSugar { override def beforeAll(): Unit = { try { @@ -190,6 +193,33 @@ class ResourceProfileSuite extends SparkFunSuite { assert(immrprof.isCoresLimitKnown == true) } + test("tasks and limit resource for task resource profile") { + val sparkConf = new SparkConf().setMaster("spark://testing") + .set(EXECUTOR_CORES, 2) + .set("spark.dynamicAllocation.enabled", "false") + .set("spark.executor.resource.gpu.amount", "2") + .set("spark.executor.resource.gpu.discoveryScript", "myscript") + + withMockSparkEnv(sparkConf) { + val rpBuilder1 = new ResourceProfileBuilder() + val rp1 = rpBuilder1 + .require(new TaskResourceRequests().resource("gpu", 1)) + .build() + assert(rp1.isInstanceOf[TaskResourceProfile]) + assert(rp1.limitingResource(sparkConf) == ResourceProfile.CPUS) + assert(rp1.maxTasksPerExecutor(sparkConf) == 2) + assert(rp1.isCoresLimitKnown) + + val rpBuilder2 = new ResourceProfileBuilder() + val rp2 = rpBuilder2 + .require(new TaskResourceRequests().resource("gpu", 2)) + .build() + assert(rp1.isInstanceOf[TaskResourceProfile]) + assert(rp2.limitingResource(sparkConf) == "gpu") + assert(rp2.maxTasksPerExecutor(sparkConf) == 1) + assert(rp2.isCoresLimitKnown) + } + } test("Create ResourceProfile") { val rprof = new ResourceProfileBuilder() @@ -257,6 +287,22 @@ class ResourceProfileSuite extends SparkFunSuite { assert(rprof.resourcesEqual(rprof2), "resource profile resourcesEqual not working") } + test("test TaskResourceProfiles equal") { + val rprofBuilder = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + rprofBuilder.require(taskReq) + val rprof = rprofBuilder.build() + + val taskReq1 = new TaskResourceRequests().resource("gpu", 1) + val rprof1 = new ResourceProfile(Map.empty, taskReq1.requests) + assert(!rprof.resourcesEqual(rprof1), + "resource profiles having different types should not equal") + + val taskReq2 = new TaskResourceRequests().resource("gpu", 1) + val rprof2 = new TaskResourceProfile(taskReq2.requests) + assert(rprof.resourcesEqual(rprof2), "task resource profile resourcesEqual not working") + } + test("Test ExecutorResourceRequests memory helpers") { val rprof = new ResourceProfileBuilder() val ereqs = new ExecutorResourceRequests() @@ -314,7 +360,7 @@ class ResourceProfileSuite extends SparkFunSuite { // Update this if new resource type added assert(ResourceProfile.allSupportedExecutorResources.size === 5, "Executor resources should have 5 supported resources") - assert(ResourceProfile.getCustomExecutorResources(rprof.build).size === 1, + assert(rprof.build().getCustomExecutorResources().size === 1, "Executor resources should have 1 custom resource") } @@ -327,7 +373,18 @@ class ResourceProfileSuite extends SparkFunSuite { .memoryOverhead("2048").pysparkMemory("1024").offHeapMemory("3072") rprof.require(taskReq).require(eReq) - assert(ResourceProfile.getCustomTaskResources(rprof.build).size === 1, + assert(rprof.build().getCustomTaskResources().size === 1, "Task resources should have 1 custom resource") } + + private def withMockSparkEnv(conf: SparkConf)(f: => Unit): Unit = { + val previousEnv = SparkEnv.get + val mockEnv = mock[SparkEnv] + when(mockEnv.conf).thenReturn(conf) + SparkEnv.set(mockEnv) + + try f finally { + SparkEnv.set(previousEnv) + } + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 847e0622213d0..19a9af86afbc1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -40,7 +40,7 @@ import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests import org.apache.spark.network.shuffle.ExternalBlockStoreClient import org.apache.spark.rdd.{DeterministicLevel, RDD} -import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, TaskResourceRequests} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, TaskResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.local.LocalSchedulerBackend @@ -3424,6 +3424,23 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti assert(mergedRp.getExecutorCores.get == 4) } + test("test merge task resource profiles") { + conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") + // Ensure the initialization of SparkEnv + sc + + val treqs1 = new TaskResourceRequests().cpus(1) + val rp1 = new TaskResourceProfile(treqs1.requests) + val treqs2 = new TaskResourceRequests().cpus(1) + val rp2 = new TaskResourceProfile(treqs2.requests) + val treqs3 = new TaskResourceRequests().cpus(2) + val rp3 = new TaskResourceProfile(treqs3.requests) + val mergedRp = scheduler.mergeResourceProfilesForStage(HashSet(rp1, rp2, rp3)) + + assert(mergedRp.isInstanceOf[TaskResourceProfile]) + assert(mergedRp.getTaskCpus.get == 2) + } + /** * Checks the DAGScheduler's internal logic for traversing an RDD DAG by making sure that * getShuffleDependenciesAndResourceProfiles correctly returns the direct shuffle dependencies diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 869a72324376b..4e9e9755e8502 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -33,7 +33,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.config -import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, TaskResourceRequests} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, TaskResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.util.{Clock, ManualClock, ThreadUtils} @@ -1833,6 +1833,101 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(2 == taskDescriptions.head.resources(GPU).addresses.size) } + test("Scheduler works with task resource profiles") { + val taskCpus = 1 + val taskGpus = 1 + val executorGpus = 4 + val executorCpus = 4 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString + ) + + val treqs = new TaskResourceRequests().cpus(2).resource(GPU, 2) + val rp = new TaskResourceProfile(treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + val taskSet = FakeTask.createTaskSet(3) + val rpTaskSet = FakeTask.createTaskSet(5, stageId = 1, stageAttemptId = 0, + priority = 0, rpId = rp.id) + + val resources0 = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) + val resources1 = Map(GPU -> ArrayBuffer("4", "5", "6", "7")) + + val workerOffers = + IndexedSeq(WorkerOffer("executor0", "host0", 4, None, resources0), + WorkerOffer("executor1", "host1", 4, None, resources1)) + + taskScheduler.submitTasks(taskSet) + taskScheduler.submitTasks(rpTaskSet) + // should have 3 for default profile and 2 for additional resource profile + var taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(5 === taskDescriptions.length) + var has2Gpus = 0 + var has1Gpu = 0 + for (tDesc <- taskDescriptions) { + assert(tDesc.resources.contains(GPU)) + if (tDesc.resources(GPU).addresses.size == 2) { + has2Gpus += 1 + } + if (tDesc.resources(GPU).addresses.size == 1) { + has1Gpu += 1 + } + } + assert(has2Gpus == 2) + assert(has1Gpu == 3) + + val resources3 = Map(GPU -> ArrayBuffer("8", "9", "10", "11")) + + // clear the first 2 worker offers so they don't have any room and add a third + // for the resource profile + val workerOffers3 = IndexedSeq( + WorkerOffer("executor0", "host0", 0, None, Map.empty), + WorkerOffer("executor1", "host1", 0, None, Map.empty), + WorkerOffer("executor2", "host2", 4, None, resources3)) + taskDescriptions = taskScheduler.resourceOffers(workerOffers3).flatten + assert(2 === taskDescriptions.length) + assert(taskDescriptions.head.resources.contains(GPU)) + assert(2 == taskDescriptions.head.resources(GPU).addresses.size) + } + + test("Calculate available tasks slots for task resource profiles") { + val taskCpus = 1 + val taskGpus = 1 + val executorGpus = 4 + val executorCpus = 4 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString + ) + + val treqs = new TaskResourceRequests().cpus(2).resource(GPU, 2) + val rp = new TaskResourceProfile(treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val resources0 = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) + val resources1 = Map(GPU -> ArrayBuffer("4", "5", "6", "7")) + + val workerOffers = + IndexedSeq(WorkerOffer("executor0", "host0", 4, None, resources0), + WorkerOffer("executor1", "host1", 4, None, resources1)) + val availableResourcesAmount = workerOffers.map(_.resources).map { resourceMap => + // available addresses already takes into account if there are fractional + // task resource requests + resourceMap.map { case (name, addresses) => (name, addresses.length) } + } + + val taskSlotsForRp = TaskSchedulerImpl.calculateAvailableSlots( + taskScheduler, taskScheduler.conf, rp.id, workerOffers.map(_.resourceProfileId).toArray, + workerOffers.map(_.cores).toArray, availableResourcesAmount.toArray) + assert(taskSlotsForRp === 4) + } + private def setupSchedulerForDecommissionTests(clock: Clock, numTasks: Int): TaskSchedulerImpl = { // one task per host val numHosts = numTasks diff --git a/docs/configuration.md b/docs/configuration.md index 55e595ad30158..ffd36209e2da7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3243,9 +3243,9 @@ See your cluster manager specific page for requirements and details on each of - # Stage Level Scheduling Overview The stage level scheduling feature allows users to specify task and executor resource requirements at the stage level. This allows for different stages to run with executors that have different resources. A prime example of this is one ETL stage runs with executors with just CPUs, the next stage is an ML stage that needs GPUs. Stage level scheduling allows for user to request different executors that have GPUs when the ML stage runs rather then having to acquire executors with GPUs at the start of the application and them be idle while the ETL stage is being run. -This is only available for the RDD API in Scala, Java, and Python. It is available on YARN, Kubernetes and Standalone when dynamic allocation is enabled. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page or [Kubernetes](running-on-kubernetes.html#stage-level-scheduling-overview) page or [Standalone](spark-standalone.html#stage-level-scheduling-overview) page for more implementation details. +This is only available for the RDD API in Scala, Java, and Python. It is available on YARN, Kubernetes and Standalone when dynamic allocation is enabled. When dynamic allocation is disabled, it allows users to specify different task resource requirements at stage level, and this is supported on Standalone cluster right now. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page or [Kubernetes](running-on-kubernetes.html#stage-level-scheduling-overview) page or [Standalone](spark-standalone.html#stage-level-scheduling-overview) page for more implementation details. -See the `RDD.withResources` and `ResourceProfileBuilder` API's for using this feature. The current implementation acquires new executors for each `ResourceProfile` created and currently has to be an exact match. Spark does not try to fit tasks into an executor that require a different ResourceProfile than the executor was created with. Executors that are not in use will idle timeout with the dynamic allocation logic. The default configuration for this feature is to only allow one ResourceProfile per stage. If the user associates more then 1 ResourceProfile to an RDD, Spark will throw an exception by default. See config `spark.scheduler.resource.profileMergeConflicts` to control that behavior. The current merge strategy Spark implements when `spark.scheduler.resource.profileMergeConflicts` is enabled is a simple max of each resource within the conflicting ResourceProfiles. Spark will create a new ResourceProfile with the max of each of the resources. +See the `RDD.withResources` and `ResourceProfileBuilder` API's for using this feature. When dynamic allocation is disabled, tasks with different task resource requirements will share executors with `DEFAULT_RESOURCE_PROFILE`. While when dynamic allocation is enabled, the current implementation acquires new executors for each `ResourceProfile` created and currently has to be an exact match. Spark does not try to fit tasks into an executor that require a different ResourceProfile than the executor was created with. Executors that are not in use will idle timeout with the dynamic allocation logic. The default configuration for this feature is to only allow one ResourceProfile per stage. If the user associates more then 1 ResourceProfile to an RDD, Spark will throw an exception by default. See config `spark.scheduler.resource.profileMergeConflicts` to control that behavior. The current merge strategy Spark implements when `spark.scheduler.resource.profileMergeConflicts` is enabled is a simple max of each resource within the conflicting ResourceProfiles. Spark will create a new ResourceProfile with the max of each of the resources. # Push-based shuffle overview diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 559e3bca6c934..b431752f166be 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -467,7 +467,9 @@ worker during one single schedule iteration. # Stage Level Scheduling Overview -Stage level scheduling is supported on Standalone when dynamic allocation is enabled. Currently, when the Master allocates executors for one application, it will schedule based on the order of the ResourceProfile ids for multiple ResourceProfiles. The ResourceProfile with smaller id will be scheduled firstly. Normally this won’t matter as Spark finishes one stage before starting another one, the only case this might have an affect is in a job server type scenario, so its something to keep in mind. For scheduling, we will only take executor memory and executor cores from built-in executor resources and all other custom resources from a ResourceProfile, other built-in executor resources such as offHeap and memoryOverhead won't take any effect. The base default profile will be created based on the spark configs when you submit an application. Executor memory and executor cores from the base default profile can be propagated to custom ResourceProfiles, but all other custom resources can not be propagated. +Stage level scheduling is supported on Standalone: +- When dynamic allocation is disabled: It allows users to specify different task resource requirements at the stage level and will use the same executors requested at startup. +- When dynamic allocation is enabled: Currently, when the Master allocates executors for one application, it will schedule based on the order of the ResourceProfile ids for multiple ResourceProfiles. The ResourceProfile with smaller id will be scheduled firstly. Normally this won’t matter as Spark finishes one stage before starting another one, the only case this might have an affect is in a job server type scenario, so its something to keep in mind. For scheduling, we will only take executor memory and executor cores from built-in executor resources and all other custom resources from a ResourceProfile, other built-in executor resources such as offHeap and memoryOverhead won't take any effect. The base default profile will be created based on the spark configs when you submit an application. Executor memory and executor cores from the base default profile can be propagated to custom ResourceProfiles, but all other custom resources can not be propagated. ## Caveats From 728f7e9841cd830bf451eddd4b5297ab2d92e5c1 Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Fri, 30 Sep 2022 09:42:29 +0800 Subject: [PATCH 2/8] [SPARK-40314][SQL][PYTHON] Add scala and python bindings for inline and inline_outer ### What changes were proposed in this pull request? Adds Scala and Python bindings for SQL functions inline and inline_outer ### Why are the changes needed? Currently these functions can only be used via SQL or through `expr` functions. This makes it a little easier to use them with the DataFrame APIs. ### Does this PR introduce _any_ user-facing change? Exposes new functions directly instead of only through SQL. ### How was this patch tested? Updated existing inline tests to use the new Scala binding instead of being called through SQL expressions Closes #37770 from Kimahriman/inline-bindings. Authored-by: Adam Binford Signed-off-by: Ruifeng Zheng --- .../reference/pyspark.sql/functions.rst | 2 + python/pyspark/sql/functions.py | 76 +++++++++++++++++++ python/pyspark/sql/tests/test_functions.py | 16 ++++ .../org/apache/spark/sql/functions.scala | 17 +++++ .../spark/sql/GeneratorFunctionSuite.scala | 32 ++++---- 5 files changed, 127 insertions(+), 16 deletions(-) diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index 027babbf57d6c..5a64845598ea5 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -176,6 +176,8 @@ Collection Functions explode_outer posexplode posexplode_outer + inline + inline_outer get get_json_object json_tuple diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 9b4b5f15e9e9f..f01379afd6ef9 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -6956,6 +6956,41 @@ def posexplode(col: "ColumnOrName") -> Column: return _invoke_function_over_columns("posexplode", col) +def inline(col: "ColumnOrName") -> Column: + """ + Explodes an array of structs into a table. + + .. versionadded:: 3.4.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or str + input column of values to explode. + + Returns + ------- + :class:`~pyspark.sql.Column` + generator expression with the inline exploded result. + + See Also + -------- + :meth:`explode` + + Examples + -------- + >>> from pyspark.sql import Row + >>> df = spark.createDataFrame([Row(structlist=[Row(a=1, b=2), Row(a=3, b=4)])]) + >>> df.select(inline(df.structlist)).show() + +---+---+ + | a| b| + +---+---+ + | 1| 2| + | 3| 4| + +---+---+ + """ + return _invoke_function_over_columns("inline", col) + + def explode_outer(col: "ColumnOrName") -> Column: """ Returns a new row for each element in the given array or map. @@ -7049,6 +7084,47 @@ def posexplode_outer(col: "ColumnOrName") -> Column: return _invoke_function_over_columns("posexplode_outer", col) +def inline_outer(col: "ColumnOrName") -> Column: + """ + Explodes an array of structs into a table. + Unlike inline, if the array is null or empty then null is produced for each nested column. + + .. versionadded:: 3.4.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or str + input column of values to explode. + + Returns + ------- + :class:`~pyspark.sql.Column` + generator expression with the inline exploded result. + + See Also + -------- + :meth:`explode_outer` + :meth:`inline` + + Examples + -------- + >>> from pyspark.sql import Row + >>> df = spark.createDataFrame([ + ... Row(id=1, structlist=[Row(a=1, b=2), Row(a=3, b=4)]), + ... Row(id=2, structlist=[]) + ... ]) + >>> df.select('id', inline_outer(df.structlist)).show() + +---+----+----+ + | id| a| b| + +---+----+----+ + | 1| 1| 2| + | 1| 3| 4| + | 2|null|null| + +---+----+----+ + """ + return _invoke_function_over_columns("inline_outer", col) + + def get_json_object(col: "ColumnOrName", path: str) -> Column: """ Extracts json object from a json string based on json `path` specified, and returns json string diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index a9bcdd0ba65a1..9edad04622766 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -145,6 +145,22 @@ def test_explode(self): result = [tuple(x) for x in data.select(explode_outer("mapfield")).collect()] self.assertEqual(result, [("a", "b"), (None, None), (None, None)]) + def test_inline(self): + from pyspark.sql.functions import inline, inline_outer + + d = [ + Row(structlist=[Row(b=1, c=2), Row(b=3, c=4)]), + Row(structlist=[Row(b=None, c=5), None]), + Row(structlist=[]), + ] + data = self.spark.createDataFrame(d) + + result = [tuple(x) for x in data.select(inline(data.structlist)).collect()] + self.assertEqual(result, [(1, 2), (3, 4), (None, 5), (None, None)]) + + result = [tuple(x) for x in data.select(inline_outer(data.structlist)).collect()] + self.assertEqual(result, [(1, 2), (3, 4), (None, 5), (None, None), (None, None)]) + def test_basic_functions(self): rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) df = self.spark.read.json(rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index f5d284f143f7f..620e1c6072172 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -4345,6 +4345,23 @@ object functions { */ def posexplode_outer(e: Column): Column = withExpr { GeneratorOuter(PosExplode(e.expr)) } + /** + * Creates a new row for each element in the given array of structs. + * + * @group collection_funcs + * @since 3.4.0 + */ + def inline(e: Column): Column = withExpr { Inline(e.expr) } + + /** + * Creates a new row for each element in the given array of structs. + * Unlike inline, if the array is null or empty then null is produced for each nested column. + * + * @group collection_funcs + * @since 3.4.0 + */ + def inline_outer(e: Column): Column = withExpr { GeneratorOuter(Inline(e.expr)) } + /** * Extracts json object from a json string based on json path specified, and returns json string * of the extracted json object. It will return null if the input json string is invalid. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index 648f9cac216ec..25231fdecba28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -219,20 +219,21 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { test("inline raises exception on array of null type") { val m = intercept[AnalysisException] { - spark.range(2).selectExpr("inline(array())") + spark.range(2).select(inline(array())) }.getMessage assert(m.contains("data type mismatch")) } test("inline with empty table") { checkAnswer( - spark.range(0).selectExpr("inline(array(struct(10, 100)))"), + spark.range(0).select(inline(array(struct(lit(10), lit(100))))), Nil) } test("inline on literal") { checkAnswer( - spark.range(2).selectExpr("inline(array(struct(10, 100), struct(20, 200), struct(30, 300)))"), + spark.range(2).select(inline(array(struct(lit(10), lit(100)), struct(lit(20), lit(200)), + struct(lit(30), lit(300))))), Row(10, 100) :: Row(20, 200) :: Row(30, 300) :: Row(10, 100) :: Row(20, 200) :: Row(30, 300) :: Nil) } @@ -241,39 +242,39 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((1, 2)).toDF("a", "b") checkAnswer( - df.selectExpr("inline(array(struct(a), struct(a)))"), + df.select(inline(array(struct('a), struct('a)))), Row(1) :: Row(1) :: Nil) checkAnswer( - df.selectExpr("inline(array(struct(a, b), struct(a, b)))"), + df.select(inline(array(struct('a, 'b), struct('a, 'b)))), Row(1, 2) :: Row(1, 2) :: Nil) // Spark think [struct, struct] is heterogeneous due to name difference. val m = intercept[AnalysisException] { - df.selectExpr("inline(array(struct(a), struct(b)))") + df.select(inline(array(struct('a), struct('b)))) }.getMessage assert(m.contains("data type mismatch")) checkAnswer( - df.selectExpr("inline(array(struct(a), named_struct('a', b)))"), + df.select(inline(array(struct('a), struct('b.alias("a"))))), Row(1) :: Row(2) :: Nil) // Spark think [struct, struct] is heterogeneous due to name difference. val m2 = intercept[AnalysisException] { - df.selectExpr("inline(array(struct(a), struct(2)))") + df.select(inline(array(struct('a), struct(lit(2))))) }.getMessage assert(m2.contains("data type mismatch")) checkAnswer( - df.selectExpr("inline(array(struct(a), named_struct('a', 2)))"), + df.select(inline(array(struct('a), struct(lit(2).alias("a"))))), Row(1) :: Row(2) :: Nil) checkAnswer( - df.selectExpr("struct(a)").selectExpr("inline(array(*))"), + df.select(struct('a)).select(inline(array("*"))), Row(1) :: Nil) checkAnswer( - df.selectExpr("array(struct(a), named_struct('a', b))").selectExpr("inline(*)"), + df.select(array(struct('a), struct('b.alias("a")))).selectExpr("inline(*)"), Row(1) :: Row(2) :: Nil) } @@ -282,11 +283,11 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df2 = df.select( when($"col1" === 1, null).otherwise(array(struct($"col1", $"col2"))).as("col1")) checkAnswer( - df2.selectExpr("inline(col1)"), + df2.select(inline('col1)), Row(3, "4") :: Row(5, "6") :: Nil ) checkAnswer( - df2.selectExpr("inline_outer(col1)"), + df2.select(inline_outer('col1)), Row(null, null) :: Row(3, "4") :: Row(5, "6") :: Nil ) } @@ -405,14 +406,13 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { |) |as tbl(a, b) """.stripMargin) - df.createOrReplaceTempView("t1") checkAnswer( - sql("select inline(b) from t1"), + df.select(inline('b)), Row(0, 1) :: Row(null, null) :: Row(2, 3) :: Row(null, null) :: Nil) checkAnswer( - sql("select a, inline(b) from t1"), + df.select('a, inline('b)), Row(1, 0, 1) :: Row(1, null, null) :: Row(1, 2, 3) :: Row(1, null, null) :: Nil) } From e617503c3f06be9eea0af529bab7984fc07e87a2 Mon Sep 17 00:00:00 2001 From: itholic Date: Fri, 30 Sep 2022 09:45:57 +0800 Subject: [PATCH 3/8] [SPARK-40589][PS][TEST] Fix test for `DataFrame.corr_with` skip the pandas regression ### What changes were proposed in this pull request? This PR proposes to skip the `DataFrame.corr_with` test when the `other` is `pyspark.pandas.Series` and the `method` is "spearman" or "pearson", since there is regression in pandas 1.5.0 for that cases. ### Why are the changes needed? There are some regressions in pandas 1.5.0, so we're not going to match the behavior for those cases. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually tested with pandas 1.5.0, confirmed the test pass. Closes #38031 from itholic/SPARK-40589. Authored-by: itholic Signed-off-by: Ruifeng Zheng --- python/pyspark/pandas/tests/test_dataframe.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/python/pyspark/pandas/tests/test_dataframe.py b/python/pyspark/pandas/tests/test_dataframe.py index 5da0974c9063d..dfac3c6d1b8e9 100644 --- a/python/pyspark/pandas/tests/test_dataframe.py +++ b/python/pyspark/pandas/tests/test_dataframe.py @@ -6076,7 +6076,14 @@ def test_corrwith(self): def _test_corrwith(self, psdf, psobj): pdf = psdf.to_pandas() pobj = psobj.to_pandas() - for method in ["pearson", "spearman", "kendall"]: + # Regression in pandas 1.5.0 when other is Series and method is "pearson" or "spearman" + # See https://github.com/pandas-dev/pandas/issues/48826 for the reported issue, + # and https://github.com/pandas-dev/pandas/pull/46174 for the initial PR that causes. + if LooseVersion(pd.__version__) >= LooseVersion("1.5.0") and isinstance(pobj, pd.Series): + methods = ["kendall"] + else: + methods = ["pearson", "spearman", "kendall"] + for method in methods: for drop in [True, False]: p_corr = pdf.corrwith(pobj, drop=drop, method=method) ps_corr = psdf.corrwith(psobj, drop=drop, method=method) From 77f5c8442b19074eaf95c92495d628969d9225d8 Mon Sep 17 00:00:00 2001 From: itholic Date: Fri, 30 Sep 2022 11:04:13 +0900 Subject: [PATCH 4/8] [SPARK-40577][PS] Fix `CategoricalIndex.append` to match pandas 1.5.0 ### What changes were proposed in this pull request? The PR proposes to fix `CategoricalIndex.append` to match the behavior with pandas. ### Why are the changes needed? Because the current behavior is different from pandas 1.5.0. ### Does this PR introduce _any_ user-facing change? The behavior of API is changed as below: **Before** ```python >>> psidx1 = ps.CategoricalIndex(["x", "y", "z"], categories=["z", "y", "x", "w"]) >>> psidx3 = ps.Index(["y", "x", "w", "z"]) >>> psidx1.append(psidx3.astype("category")) CategoricalIndex(['x', 'y', 'z', 'y', 'x', 'w', 'z'], categories=['z', 'y', 'x', 'w'], ordered=False, dtype='category') ``` **After** ```python >>> psidx1 = ps.CategoricalIndex(["x", "y", "z"], categories=["z", "y", "x", "w"]) >>> psidx3 = ps.Index(["y", "x", "w", "z"]) >>> psidx1.append(psidx3.astype("category")) CategoricalIndex(['x', 'y', 'z', 'x', 'y', 'z', 'w'], categories=['z', 'y', 'x', 'w'], ordered=False, dtype='category') ``` ### How was this patch tested? Manually check the existing test is passed with pandas 1.5.0. Closes #38015 from itholic/SPARK-40577. Authored-by: itholic Signed-off-by: Hyukjin Kwon --- python/pyspark/pandas/indexes/base.py | 4 ++++ .../pyspark/pandas/tests/indexes/test_category.py | 15 ++++++++++++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/python/pyspark/pandas/indexes/base.py b/python/pyspark/pandas/indexes/base.py index 009657880ee49..ccdf405326b6d 100644 --- a/python/pyspark/pandas/indexes/base.py +++ b/python/pyspark/pandas/indexes/base.py @@ -1896,6 +1896,7 @@ def append(self, other: "Index") -> "Index": ) """ from pyspark.pandas.indexes.multi import MultiIndex + from pyspark.pandas.indexes.category import CategoricalIndex if isinstance(self, MultiIndex) != isinstance(other, MultiIndex): raise NotImplementedError( @@ -1907,6 +1908,9 @@ def append(self, other: "Index") -> "Index": ) index_fields = self._index_fields_for_union_like(other, func_name="append") + # Since pandas 1.5.0, the order of category matters. + if isinstance(other, CategoricalIndex): + other = other.reorder_categories(self.categories.to_list()) sdf_self = self._internal.spark_frame.select(self._internal.index_spark_columns) sdf_other = other._internal.spark_frame.select(other._internal.index_spark_columns) diff --git a/python/pyspark/pandas/tests/indexes/test_category.py b/python/pyspark/pandas/tests/indexes/test_category.py index e50b816ae10f6..ba737eb520f06 100644 --- a/python/pyspark/pandas/tests/indexes/test_category.py +++ b/python/pyspark/pandas/tests/indexes/test_category.py @@ -226,9 +226,18 @@ def test_append(self): psidx3 = ps.from_pandas(pidx3) self.assert_eq(psidx1.append(psidx2), pidx1.append(pidx2)) - self.assert_eq( - psidx1.append(psidx3.astype("category")), pidx1.append(pidx3.astype("category")) - ) + if LooseVersion(pd.__version__) >= LooseVersion("1.5.0"): + self.assert_eq( + psidx1.append(psidx3.astype("category")), pidx1.append(pidx3.astype("category")) + ) + else: + expected_result = ps.CategoricalIndex( + ["x", "y", "z", "y", "x", "w", "z"], + categories=["z", "y", "x", "w"], + ordered=False, + dtype="category", + ) + self.assert_eq(psidx1.append(psidx3.astype("category")), expected_result) # TODO: append non-categorical or categorical with a different category self.assertRaises(NotImplementedError, lambda: psidx1.append(psidx3)) From 7bcf4b9b8d0935d0976a457f3ece4de174c56d94 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 30 Sep 2022 13:44:41 +0800 Subject: [PATCH 5/8] [SPARK-40606][PS][TEST] Eliminate `to_pandas` warnings in test ### What changes were proposed in this pull request? Eliminate `to_pandas` warnings in test, by changing `to_pandas()` to `_to_pandas()` ### Why are the changes needed? when a test containing `to_pandas` failed, it may print tons of warnings, for example, in https://github.com/zhengruifeng/spark/actions/runs/3142284988/jobs/5106178199 it printed this warning from line 1243 to line 3985: image ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? updated tests Closes #38042 from zhengruifeng/ps_test_to_pandas. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../tests/data_type_ops/test_binary_ops.py | 2 +- .../tests/data_type_ops/test_boolean_ops.py | 2 +- .../data_type_ops/test_categorical_ops.py | 2 +- .../tests/data_type_ops/test_complex_ops.py | 2 +- .../tests/data_type_ops/test_date_ops.py | 2 +- .../tests/data_type_ops/test_datetime_ops.py | 2 +- .../tests/data_type_ops/test_null_ops.py | 2 +- .../tests/data_type_ops/test_num_ops.py | 6 +-- .../tests/data_type_ops/test_string_ops.py | 4 +- .../tests/data_type_ops/test_timedelta_ops.py | 2 +- .../tests/data_type_ops/test_udt_ops.py | 2 +- .../pyspark/pandas/tests/indexes/test_base.py | 16 +++--- .../tests/plot/test_frame_plot_plotly.py | 2 +- .../tests/plot/test_series_plot_matplotlib.py | 2 +- .../tests/plot/test_series_plot_plotly.py | 6 +-- python/pyspark/pandas/tests/test_dataframe.py | 52 ++++++++++--------- .../pandas/tests/test_dataframe_spark_io.py | 8 +-- .../pandas/tests/test_default_index.py | 2 +- python/pyspark/pandas/tests/test_groupby.py | 6 +-- python/pyspark/pandas/tests/test_indexing.py | 4 +- .../pandas/tests/test_ops_on_diff_frames.py | 8 +-- python/pyspark/pandas/tests/test_repr.py | 48 ++++++++--------- python/pyspark/pandas/tests/test_series.py | 4 +- .../pandas/tests/test_series_datetime.py | 6 +-- 24 files changed, 97 insertions(+), 95 deletions(-) diff --git a/python/pyspark/pandas/tests/data_type_ops/test_binary_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_binary_ops.py index 56bc115e5d79a..7135800bd9ba6 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_binary_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_binary_ops.py @@ -152,7 +152,7 @@ def test_from_to_pandas(self): data = [b"1", b"2", b"3"] pser = pd.Series(data) psser = ps.Series(data) - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py index fc59f6bbbb362..73761202267b7 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py @@ -732,7 +732,7 @@ def test_from_to_pandas(self): data = [True, True, False, None] pser = pd.Series(data, dtype="boolean") psser = ps.Series(data, dtype="boolean") - self.check_extension(pser, psser.to_pandas()) + self.check_extension(pser, psser._to_pandas()) self.check_extension(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_categorical_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_categorical_ops.py index 378e6e894f8d8..992e3ed70fbff 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_categorical_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_categorical_ops.py @@ -166,7 +166,7 @@ def test_from_to_pandas(self): data = [1, "x", "y"] pser = pd.Series(data, dtype="category") psser = ps.Series(data, dtype="category") - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_complex_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_complex_ops.py index 9bce1fc4b1070..bbdf837ce2904 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_complex_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_complex_ops.py @@ -239,7 +239,7 @@ def test_from_to_pandas(self): pdf, psdf = self.array_pdf, self.array_psdf for col in self.array_df_cols: pser, psser = pdf[col], psdf[col] - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py index d3ce86da0a46a..b457ab2cc85a4 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py @@ -176,7 +176,7 @@ def test_from_to_pandas(self): data = [datetime.date(1994, 1, 31), datetime.date(1994, 2, 1), datetime.date(1994, 2, 2)] pser = pd.Series(data) psser = ps.Series(data) - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_datetime_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_datetime_ops.py index 04aaed811ff8a..de9c6acb2cd15 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_datetime_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_datetime_ops.py @@ -180,7 +180,7 @@ def test_from_to_pandas(self): data = pd.date_range("1994-1-31 10:30:15", periods=3, freq="M") pser = pd.Series(data) psser = ps.Series(data) - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_null_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_null_ops.py index df3f40defff24..fc6cdd1a436b2 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_null_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_null_ops.py @@ -111,7 +111,7 @@ def test_from_to_pandas(self): data = [None, None, None] pser = pd.Series(data) psser = ps.Series(data) - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py index d897d7735e48d..cb678ff585e4d 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py @@ -320,7 +320,7 @@ def test_from_to_pandas(self): pdf, psdf = self.pdf, self.psdf for col in self.numeric_df_cols: pser, psser = pdf[col], psdf[col] - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): @@ -464,7 +464,7 @@ def intergral_extension_pser_psser_pairs(self): def test_from_to_pandas(self): for pser, psser in self.intergral_extension_pser_psser_pairs: - self.check_extension(pser, psser.to_pandas()) + self.check_extension(pser, psser._to_pandas()) self.check_extension(ps.from_pandas(pser), psser) def test_isnull(self): @@ -607,7 +607,7 @@ def fractional_extension_pser_psser_pairs(self): def test_from_to_pandas(self): for pser, psser in self.fractional_extension_pser_psser_pairs: - self.check_extension(pser, psser.to_pandas()) + self.check_extension(pser, psser._to_pandas()) self.check_extension(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py index 572ea7688cb7f..cc448dc42d6a9 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py @@ -160,7 +160,7 @@ def test_from_to_pandas(self): data = ["x", "y", "z"] pser = pd.Series(data) psser = ps.Series(data) - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): @@ -275,7 +275,7 @@ def test_from_to_pandas(self): data = ["x", "y", "z", None] pser = pd.Series(data, dtype="string") psser = ps.Series(data, dtype="string") - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_timedelta_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_timedelta_ops.py index 3d8b7d9e884af..eeaba4d277816 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_timedelta_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_timedelta_ops.py @@ -139,7 +139,7 @@ def test_from_to_pandas(self): data = [timedelta(1), timedelta(microseconds=2)] pser = pd.Series(data) psser = ps.Series(data) - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py index a789a1b249864..81767af76f95b 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py @@ -126,7 +126,7 @@ def test_from_to_pandas(self): sparse_vector = SparseVector(len(sparse_values), sparse_values) pser = pd.Series([sparse_vector]) psser = ps.Series([sparse_vector]) - self.assert_eq(pser, psser.to_pandas()) + self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) def test_isnull(self): diff --git a/python/pyspark/pandas/tests/indexes/test_base.py b/python/pyspark/pandas/tests/indexes/test_base.py index 58e27263af8dc..9ca31923d5c0f 100644 --- a/python/pyspark/pandas/tests/indexes/test_base.py +++ b/python/pyspark/pandas/tests/indexes/test_base.py @@ -380,11 +380,11 @@ def test_index_unique(self): # here the output is different than pandas in terms of order expected = [0, 1, 3, 5, 6, 8, 9] - self.assert_eq(expected, sorted(psidx.unique().to_pandas())) - self.assert_eq(expected, sorted(psidx.unique(level=0).to_pandas())) + self.assert_eq(expected, sorted(psidx.unique()._to_pandas())) + self.assert_eq(expected, sorted(psidx.unique(level=0)._to_pandas())) expected = [1, 2, 4, 6, 7, 9, 10] - self.assert_eq(expected, sorted((psidx + 1).unique().to_pandas())) + self.assert_eq(expected, sorted((psidx + 1).unique()._to_pandas())) with self.assertRaisesRegex(IndexError, "Too many levels*"): psidx.unique(level=1) @@ -507,7 +507,7 @@ def test_multi_index_symmetric_difference(self): self.assert_eq( midx.symmetric_difference(midx_), - midx.to_pandas().symmetric_difference(midx_.to_pandas()), + midx._to_pandas().symmetric_difference(midx_._to_pandas()), ) with self.assertRaisesRegex(NotImplementedError, "Doesn't support*"): @@ -1356,7 +1356,7 @@ def test_monotonic(self): psdf = ps.DataFrame({"a": [-5, -4, -3, -2, -1], "b": [1, 1, 1, 1, 1]}) psdf["b"] = None psmidx = psdf.set_index(["a", "b"]).index - pmidx = psmidx.to_pandas() + pmidx = psmidx._to_pandas() self.assert_eq(psmidx.is_monotonic_increasing, pmidx.is_monotonic_increasing) self.assert_eq(psmidx.is_monotonic_decreasing, pmidx.is_monotonic_decreasing) @@ -1364,7 +1364,7 @@ def test_monotonic(self): psdf = ps.DataFrame({"a": [1, 1, 1, 1, 1], "b": ["e", "c", "b", "d", "a"]}) psdf["a"] = None psmidx = psdf.set_index(["a", "b"]).index - pmidx = psmidx.to_pandas() + pmidx = psmidx._to_pandas() self.assert_eq(psmidx.is_monotonic_increasing, pmidx.is_monotonic_increasing) self.assert_eq(psmidx.is_monotonic_decreasing, pmidx.is_monotonic_decreasing) @@ -1373,7 +1373,7 @@ def test_monotonic(self): psdf["a"] = None psdf["b"] = None psmidx = psdf.set_index(["a", "b"]).index - pmidx = psmidx.to_pandas() + pmidx = psmidx._to_pandas() self.assert_eq(psmidx.is_monotonic_increasing, pmidx.is_monotonic_increasing) self.assert_eq(psmidx.is_monotonic_decreasing, pmidx.is_monotonic_decreasing) @@ -1382,7 +1382,7 @@ def test_monotonic(self): psdf["a"] = None psdf["b"] = None psmidx = psdf.set_index(["a", "b"]).index - pmidx = psmidx.to_pandas() + pmidx = psmidx._to_pandas() self.assert_eq(psmidx.is_monotonic_increasing, pmidx.is_monotonic_increasing) self.assert_eq(psmidx.is_monotonic_decreasing, pmidx.is_monotonic_decreasing) diff --git a/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py b/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py index 2937ef1813f74..d169326b7b7fb 100644 --- a/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py +++ b/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py @@ -157,7 +157,7 @@ def check_scatter_plot(pdf, psdf, x, y, c): def test_pie_plot(self): def check_pie_plot(psdf): - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assertEqual( psdf.plot(kind="pie", y=psdf.columns[0]), express.pie(pdf, values="a", names=pdf.index), diff --git a/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py b/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py index 6124c3fd14f9a..680eee13deb11 100644 --- a/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py +++ b/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py @@ -70,7 +70,7 @@ def psdf2(self): @property def pdf2(self): - return self.psdf2.to_pandas() + return self.psdf2._to_pandas() @staticmethod def plot_to_base64(ax): diff --git a/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py b/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py index 8cb529d745e40..8a50b1829ddcc 100644 --- a/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py +++ b/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py @@ -70,7 +70,7 @@ def psdf2(self): @property def pdf2(self): - return self.psdf2.to_pandas() + return self.psdf2._to_pandas() def test_bar_plot(self): pdf = self.pdf1 @@ -111,7 +111,7 @@ def test_area_plot(self): def test_pie_plot(self): psdf = self.psdf1 - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assertEqual( psdf["a"].plot(kind="pie"), express.pie(pdf, values=pdf.columns[0], names=pdf.index), @@ -134,7 +134,7 @@ def test_pie_plot(self): # }, # index=pd.MultiIndex.from_tuples([("x", "y")] * 11), # ) - # pdf = psdf.to_pandas() + # pdf = psdf._to_pandas() # self.assertEqual( # psdf["a"].plot(kind="pie"), express.pie(pdf, values=pdf.columns[0], names=pdf.index), # ) diff --git a/python/pyspark/pandas/tests/test_dataframe.py b/python/pyspark/pandas/tests/test_dataframe.py index dfac3c6d1b8e9..166c18ba4e9de 100644 --- a/python/pyspark/pandas/tests/test_dataframe.py +++ b/python/pyspark/pandas/tests/test_dataframe.py @@ -723,7 +723,7 @@ def test_dataframe_column_level_name(self): self.assert_eq(psdf, pdf) self.assert_eq(psdf.columns.names, pdf.columns.names) - self.assert_eq(psdf.to_pandas().columns.names, pdf.columns.names) + self.assert_eq(psdf._to_pandas().columns.names, pdf.columns.names) def test_dataframe_multiindex_names_level(self): columns = pd.MultiIndex.from_tuples( @@ -738,7 +738,7 @@ def test_dataframe_multiindex_names_level(self): psdf = ps.from_pandas(pdf) self.assert_eq(psdf.columns.names, pdf.columns.names) - self.assert_eq(psdf.to_pandas().columns.names, pdf.columns.names) + self.assert_eq(psdf._to_pandas().columns.names, pdf.columns.names) psdf1 = ps.from_pandas(pdf) self.assert_eq(psdf1.columns.names, pdf.columns.names) @@ -750,13 +750,13 @@ def test_dataframe_multiindex_names_level(self): self.assert_eq(psdf["X"], pdf["X"]) self.assert_eq(psdf["X"].columns.names, pdf["X"].columns.names) - self.assert_eq(psdf["X"].to_pandas().columns.names, pdf["X"].columns.names) + self.assert_eq(psdf["X"]._to_pandas().columns.names, pdf["X"].columns.names) self.assert_eq(psdf["X"]["A"], pdf["X"]["A"]) self.assert_eq(psdf["X"]["A"].columns.names, pdf["X"]["A"].columns.names) - self.assert_eq(psdf["X"]["A"].to_pandas().columns.names, pdf["X"]["A"].columns.names) + self.assert_eq(psdf["X"]["A"]._to_pandas().columns.names, pdf["X"]["A"].columns.names) self.assert_eq(psdf[("X", "A")], pdf[("X", "A")]) self.assert_eq(psdf[("X", "A")].columns.names, pdf[("X", "A")].columns.names) - self.assert_eq(psdf[("X", "A")].to_pandas().columns.names, pdf[("X", "A")].columns.names) + self.assert_eq(psdf[("X", "A")]._to_pandas().columns.names, pdf[("X", "A")].columns.names) self.assert_eq(psdf[("X", "A", "Z")], pdf[("X", "A", "Z")]) def test_itertuples(self): @@ -866,7 +866,9 @@ def test_reset_index_with_default_index_types(self): with ps.option_context("compute.default_index_type", "distributed"): # the index is different. - self.assert_eq(psdf.reset_index().to_pandas().reset_index(drop=True), pdf.reset_index()) + self.assert_eq( + psdf.reset_index()._to_pandas().reset_index(drop=True), pdf.reset_index() + ) def test_reset_index_with_multiindex_columns(self): index = pd.MultiIndex.from_tuples( @@ -979,14 +981,14 @@ def test_repr_cache_invalidation(self): df = ps.range(10) df.__repr__() df["a"] = df["id"] - self.assertEqual(df.__repr__(), df.to_pandas().__repr__()) + self.assertEqual(df.__repr__(), df._to_pandas().__repr__()) def test_repr_html_cache_invalidation(self): # If there is any cache, inplace operations should invalidate it. df = ps.range(10) df._repr_html_() df["a"] = df["id"] - self.assertEqual(df._repr_html_(), df.to_pandas()._repr_html_()) + self.assertEqual(df._repr_html_(), df._to_pandas()._repr_html_()) def test_empty_dataframe(self): pdf = pd.DataFrame({"a": pd.Series([], dtype="i1"), "b": pd.Series([], dtype="str")}) @@ -2413,7 +2415,7 @@ def test_to_numpy(self): def test_to_pandas(self): pdf, psdf = self.df_pair - self.assert_eq(psdf.to_pandas(), pdf) + self.assert_eq(psdf._to_pandas(), pdf) def test_isin(self): pdf = pd.DataFrame( @@ -2510,7 +2512,7 @@ def test_merge(self): def check(op, right_psdf=right_psdf, right_pdf=right_pdf): k_res = op(left_psdf, right_psdf) - k_res = k_res.to_pandas() + k_res = k_res._to_pandas() k_res = k_res.sort_values(by=list(k_res.columns)) k_res = k_res.reset_index(drop=True) p_res = op(left_pdf, right_pdf) @@ -2667,7 +2669,7 @@ def test_merge_same_anchor(self): def check(op, right_psdf=right_psdf, right_pdf=right_pdf): k_res = op(left_psdf, right_psdf) - k_res = k_res.to_pandas() + k_res = k_res._to_pandas() k_res = k_res.sort_values(by=list(k_res.columns)) k_res = k_res.reset_index(drop=True) p_res = op(left_pdf, right_pdf) @@ -6074,8 +6076,8 @@ def test_corrwith(self): self._test_corrwith(df_bool, df_bool.B) def _test_corrwith(self, psdf, psobj): - pdf = psdf.to_pandas() - pobj = psobj.to_pandas() + pdf = psdf._to_pandas() + pobj = psobj._to_pandas() # Regression in pandas 1.5.0 when other is Series and method is "pearson" or "spearman" # See https://github.com/pandas-dev/pandas/issues/48826 for the reported issue, # and https://github.com/pandas-dev/pandas/pull/46174 for the initial PR that causes. @@ -6504,7 +6506,7 @@ def test_describe(self): # string columns psdf = ps.DataFrame({"A": ["a", "b", "b", "c"], "B": ["d", "e", "f", "f"]}) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assert_eq(psdf.describe(), pdf.describe().astype(str)) psdf.A += psdf.A pdf.A += pdf.A @@ -6527,7 +6529,7 @@ def test_describe(self): ], } ) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() # NOTE: Set `datetime_is_numeric=True` for pandas: # FutureWarning: Treating datetime data as categorical rather than numeric in # `.describe` is deprecated and will be removed in a future version of pandas. @@ -6582,7 +6584,7 @@ def test_describe(self): ], } ) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() if LooseVersion(pd.__version__) >= LooseVersion("1.1.0"): self.assert_eq( psdf.describe().loc[["count", "mean", "min", "max"]], @@ -6635,7 +6637,7 @@ def test_describe(self): ], } ) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() if LooseVersion(pd.__version__) >= LooseVersion("1.1.0"): pandas_result = pdf.describe(datetime_is_numeric=True) pandas_result.B = pandas_result.B.astype(str) @@ -6700,7 +6702,7 @@ def test_describe(self): "c": [None, None, None], } ) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() if LooseVersion(pd.__version__) >= LooseVersion("1.1.0"): pandas_result = pdf.describe(datetime_is_numeric=True) pandas_result.b = pandas_result.b.astype(str) @@ -6741,7 +6743,7 @@ def test_describe(self): def test_describe_empty(self): # Empty DataFrame psdf = ps.DataFrame(columns=["A", "B"]) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assert_eq( psdf.describe(), pdf.describe().astype(float), @@ -6749,7 +6751,7 @@ def test_describe_empty(self): # Explicit empty DataFrame numeric only psdf = ps.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assert_eq( psdf[psdf.a != psdf.a].describe(), pdf[pdf.a != pdf.a].describe(), @@ -6757,7 +6759,7 @@ def test_describe_empty(self): # Explicit empty DataFrame string only psdf = ps.DataFrame({"a": ["a", "b", "c"], "b": ["q", "w", "e"]}) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assert_eq( psdf[psdf.a != psdf.a].describe(), pdf[pdf.a != pdf.a].describe().astype(float), @@ -6770,7 +6772,7 @@ def test_describe_empty(self): "b": [pd.Timestamp(1), pd.Timestamp(1), pd.Timestamp(1)], } ) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() # For timestamp type, we should convert NaT to None in pandas result # since pandas API on Spark doesn't support the NaT for object type. if LooseVersion(pd.__version__) >= LooseVersion("1.1.0"): @@ -6811,7 +6813,7 @@ def test_describe_empty(self): psdf = ps.DataFrame( {"a": [1, 2, 3], "b": [pd.Timestamp(1), pd.Timestamp(1), pd.Timestamp(1)]} ) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() if LooseVersion(pd.__version__) >= LooseVersion("1.1.0"): pdf_result = pdf[pdf.a != pdf.a].describe(datetime_is_numeric=True) pdf_result.b = pdf_result.b.where(pdf_result.b.notnull(), None).astype(str) @@ -6851,7 +6853,7 @@ def test_describe_empty(self): # Explicit empty DataFrame numeric & string psdf = ps.DataFrame({"a": [1, 2, 3], "b": ["a", "b", "c"]}) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assert_eq( psdf[psdf.a != psdf.a].describe(), pdf[pdf.a != pdf.a].describe(), @@ -6861,7 +6863,7 @@ def test_describe_empty(self): psdf = ps.DataFrame( {"a": ["a", "b", "c"], "b": [pd.Timestamp(1), pd.Timestamp(1), pd.Timestamp(1)]} ) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() if LooseVersion(pd.__version__) >= LooseVersion("1.1.0"): pdf_result = pdf[pdf.a != pdf.a].describe(datetime_is_numeric=True) self.assert_eq( diff --git a/python/pyspark/pandas/tests/test_dataframe_spark_io.py b/python/pyspark/pandas/tests/test_dataframe_spark_io.py index 6c9cc13e8250f..03548a35c25b2 100644 --- a/python/pyspark/pandas/tests/test_dataframe_spark_io.py +++ b/python/pyspark/pandas/tests/test_dataframe_spark_io.py @@ -57,7 +57,7 @@ def test_parquet_read(self): def check(columns): expected = pd.read_parquet(tmp, columns=columns) actual = ps.read_parquet(tmp, columns=columns) - self.assertPandasEqual(expected, actual.to_pandas()) + self.assertPandasEqual(expected, actual._to_pandas()) check(None) check(["i32", "i64"]) @@ -66,7 +66,7 @@ def check(columns): # check with pyspark patch. expected = pd.read_parquet(tmp) actual = ps.read_parquet(tmp) - self.assertPandasEqual(expected, actual.to_pandas()) + self.assertPandasEqual(expected, actual._to_pandas()) # When index columns are known pdf = self.test_pdf @@ -386,13 +386,13 @@ def test_read_orc(self): expected = data.reset_index()[data.columns] actual = ps.read_orc(path) - self.assertPandasEqual(expected, actual.to_pandas()) + self.assertPandasEqual(expected, actual._to_pandas()) # columns columns = ["i32", "i64"] expected = data.reset_index()[columns] actual = ps.read_orc(path, columns=columns) - self.assertPandasEqual(expected, actual.to_pandas()) + self.assertPandasEqual(expected, actual._to_pandas()) # index_col expected = data.set_index("i32") diff --git a/python/pyspark/pandas/tests/test_default_index.py b/python/pyspark/pandas/tests/test_default_index.py index 4193540bd70aa..fe744d127976d 100644 --- a/python/pyspark/pandas/tests/test_default_index.py +++ b/python/pyspark/pandas/tests/test_default_index.py @@ -35,7 +35,7 @@ def test_default_index_distributed_sequence(self): def test_default_index_distributed(self): with ps.option_context("compute.default_index_type", "distributed"): sdf = self.spark.range(1000) - pdf = ps.DataFrame(sdf).to_pandas() + pdf = ps.DataFrame(sdf)._to_pandas() self.assertEqual(len(set(pdf.index)), len(pdf)) diff --git a/python/pyspark/pandas/tests/test_groupby.py b/python/pyspark/pandas/tests/test_groupby.py index e01319298a52f..14e699793f318 100644 --- a/python/pyspark/pandas/tests/test_groupby.py +++ b/python/pyspark/pandas/tests/test_groupby.py @@ -1080,7 +1080,7 @@ def test_unique(self): with self.subTest(pdf=pdf): psdf = ps.from_pandas(pdf) - actual = psdf.groupby("a")["b"].unique().sort_index().to_pandas() + actual = psdf.groupby("a")["b"].unique().sort_index()._to_pandas() expect = pdf.groupby("a")["b"].unique().sort_index() self.assert_eq(len(actual), len(expect)) for act, exp in zip(actual, expect): @@ -2367,7 +2367,7 @@ def sum_with_acc_frame(x) -> ps.DataFrame[np.float64, np.float64]: actual = psdf.groupby("d").apply(sum_with_acc_frame) actual.columns = ["d", "v"] self.assert_eq( - actual.to_pandas().sort_index(), + actual._to_pandas().sort_index(), pdf.groupby("d").apply(sum).sort_index().reset_index(drop=True), ) self.assert_eq(acc.value, 2) @@ -2378,7 +2378,7 @@ def sum_with_acc_series(x) -> np.float64: return np.sum(x) self.assert_eq( - psdf.groupby("d")["v"].apply(sum_with_acc_series).to_pandas().sort_index(), + psdf.groupby("d")["v"].apply(sum_with_acc_series)._to_pandas().sort_index(), pdf.groupby("d")["v"].apply(sum).sort_index().reset_index(drop=True), ) self.assert_eq(acc.value, 4) diff --git a/python/pyspark/pandas/tests/test_indexing.py b/python/pyspark/pandas/tests/test_indexing.py index fcce93aaafba3..c939a69929bfe 100644 --- a/python/pyspark/pandas/tests/test_indexing.py +++ b/python/pyspark/pandas/tests/test_indexing.py @@ -138,10 +138,10 @@ def test_from_pandas_with_explicit_index(self): pdf = self.pdf df1 = ps.from_pandas(pdf.set_index("month")) - self.assertPandasEqual(df1.to_pandas(), pdf.set_index("month")) + self.assertPandasEqual(df1._to_pandas(), pdf.set_index("month")) df2 = ps.from_pandas(pdf.set_index(["year", "month"])) - self.assertPandasEqual(df2.to_pandas(), pdf.set_index(["year", "month"])) + self.assertPandasEqual(df2._to_pandas(), pdf.set_index(["year", "month"])) def test_limitations(self): df = self.psdf.set_index("month") diff --git a/python/pyspark/pandas/tests/test_ops_on_diff_frames.py b/python/pyspark/pandas/tests/test_ops_on_diff_frames.py index 276daa635cf6a..b7ebe77b3115b 100644 --- a/python/pyspark/pandas/tests/test_ops_on_diff_frames.py +++ b/python/pyspark/pandas/tests/test_ops_on_diff_frames.py @@ -1512,9 +1512,9 @@ def test_series_dot(self): self.assert_eq(psser.dot(psdf), pser.dot(pdf)) psser = ps.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}).b - pser = psser.to_pandas() + pser = psser._to_pandas() psdf = ps.DataFrame({"c": [7, 8, 9]}) - pdf = psdf.to_pandas() + pdf = psdf._to_pandas() self.assert_eq(psser.dot(psdf), pser.dot(pdf)) # SPARK-36968: ps.Series.dot raise "matrices are not aligned" if index is not same @@ -1881,8 +1881,8 @@ def test_corrwith(self): self._test_corrwith(self.psdf3, self.psdf4.f) def _test_corrwith(self, psdf, psobj): - pdf = psdf.to_pandas() - pobj = psobj.to_pandas() + pdf = psdf._to_pandas() + pobj = psobj._to_pandas() for drop in [True, False]: p_corr = pdf.corrwith(pobj, drop=drop) ps_corr = psdf.corrwith(psobj, drop=drop) diff --git a/python/pyspark/pandas/tests/test_repr.py b/python/pyspark/pandas/tests/test_repr.py index d5319eb23c846..271ed0a14cb73 100644 --- a/python/pyspark/pandas/tests/test_repr.py +++ b/python/pyspark/pandas/tests/test_repr.py @@ -38,87 +38,87 @@ def tearDownClass(cls): def test_repr_dataframe(self): psdf = ps.range(ReprTest.max_display_count) self.assertTrue("Showing only the first" not in repr(psdf)) - self.assert_eq(repr(psdf), repr(psdf.to_pandas())) + self.assert_eq(repr(psdf), repr(psdf._to_pandas())) psdf = ps.range(ReprTest.max_display_count + 1) self.assertTrue("Showing only the first" in repr(psdf)) self.assertTrue( - repr(psdf).startswith(repr(psdf.to_pandas().head(ReprTest.max_display_count))) + repr(psdf).startswith(repr(psdf._to_pandas().head(ReprTest.max_display_count))) ) with option_context("display.max_rows", None): psdf = ps.range(ReprTest.max_display_count + 1) - self.assert_eq(repr(psdf), repr(psdf.to_pandas())) + self.assert_eq(repr(psdf), repr(psdf._to_pandas())) def test_repr_series(self): psser = ps.range(ReprTest.max_display_count).id self.assertTrue("Showing only the first" not in repr(psser)) - self.assert_eq(repr(psser), repr(psser.to_pandas())) + self.assert_eq(repr(psser), repr(psser._to_pandas())) psser = ps.range(ReprTest.max_display_count + 1).id self.assertTrue("Showing only the first" in repr(psser)) self.assertTrue( - repr(psser).startswith(repr(psser.to_pandas().head(ReprTest.max_display_count))) + repr(psser).startswith(repr(psser._to_pandas().head(ReprTest.max_display_count))) ) with option_context("display.max_rows", None): psser = ps.range(ReprTest.max_display_count + 1).id - self.assert_eq(repr(psser), repr(psser.to_pandas())) + self.assert_eq(repr(psser), repr(psser._to_pandas())) psser = ps.range(ReprTest.max_display_count).id.rename() self.assertTrue("Showing only the first" not in repr(psser)) - self.assert_eq(repr(psser), repr(psser.to_pandas())) + self.assert_eq(repr(psser), repr(psser._to_pandas())) psser = ps.range(ReprTest.max_display_count + 1).id.rename() self.assertTrue("Showing only the first" in repr(psser)) self.assertTrue( - repr(psser).startswith(repr(psser.to_pandas().head(ReprTest.max_display_count))) + repr(psser).startswith(repr(psser._to_pandas().head(ReprTest.max_display_count))) ) with option_context("display.max_rows", None): psser = ps.range(ReprTest.max_display_count + 1).id.rename() - self.assert_eq(repr(psser), repr(psser.to_pandas())) + self.assert_eq(repr(psser), repr(psser._to_pandas())) psser = ps.MultiIndex.from_tuples( [(100 * i, i) for i in range(ReprTest.max_display_count)] ).to_series() self.assertTrue("Showing only the first" not in repr(psser)) - self.assert_eq(repr(psser), repr(psser.to_pandas())) + self.assert_eq(repr(psser), repr(psser._to_pandas())) psser = ps.MultiIndex.from_tuples( [(100 * i, i) for i in range(ReprTest.max_display_count + 1)] ).to_series() self.assertTrue("Showing only the first" in repr(psser)) self.assertTrue( - repr(psser).startswith(repr(psser.to_pandas().head(ReprTest.max_display_count))) + repr(psser).startswith(repr(psser._to_pandas().head(ReprTest.max_display_count))) ) with option_context("display.max_rows", None): psser = ps.MultiIndex.from_tuples( [(100 * i, i) for i in range(ReprTest.max_display_count + 1)] ).to_series() - self.assert_eq(repr(psser), repr(psser.to_pandas())) + self.assert_eq(repr(psser), repr(psser._to_pandas())) def test_repr_indexes(self): psidx = ps.range(ReprTest.max_display_count).index self.assertTrue("Showing only the first" not in repr(psidx)) - self.assert_eq(repr(psidx), repr(psidx.to_pandas())) + self.assert_eq(repr(psidx), repr(psidx._to_pandas())) psidx = ps.range(ReprTest.max_display_count + 1).index self.assertTrue("Showing only the first" in repr(psidx)) self.assertTrue( repr(psidx).startswith( - repr(psidx.to_pandas().to_series().head(ReprTest.max_display_count).index) + repr(psidx._to_pandas().to_series().head(ReprTest.max_display_count).index) ) ) with option_context("display.max_rows", None): psidx = ps.range(ReprTest.max_display_count + 1).index - self.assert_eq(repr(psidx), repr(psidx.to_pandas())) + self.assert_eq(repr(psidx), repr(psidx._to_pandas())) psidx = ps.MultiIndex.from_tuples([(100 * i, i) for i in range(ReprTest.max_display_count)]) self.assertTrue("Showing only the first" not in repr(psidx)) - self.assert_eq(repr(psidx), repr(psidx.to_pandas())) + self.assert_eq(repr(psidx), repr(psidx._to_pandas())) psidx = ps.MultiIndex.from_tuples( [(100 * i, i) for i in range(ReprTest.max_display_count + 1)] @@ -126,7 +126,7 @@ def test_repr_indexes(self): self.assertTrue("Showing only the first" in repr(psidx)) self.assertTrue( repr(psidx).startswith( - repr(psidx.to_pandas().to_frame().head(ReprTest.max_display_count).index) + repr(psidx._to_pandas().to_frame().head(ReprTest.max_display_count).index) ) ) @@ -134,19 +134,19 @@ def test_repr_indexes(self): psidx = ps.MultiIndex.from_tuples( [(100 * i, i) for i in range(ReprTest.max_display_count + 1)] ) - self.assert_eq(repr(psidx), repr(psidx.to_pandas())) + self.assert_eq(repr(psidx), repr(psidx._to_pandas())) def test_html_repr(self): psdf = ps.range(ReprTest.max_display_count) self.assertTrue("Showing only the first" not in psdf._repr_html_()) - self.assertEqual(psdf._repr_html_(), psdf.to_pandas()._repr_html_()) + self.assertEqual(psdf._repr_html_(), psdf._to_pandas()._repr_html_()) psdf = ps.range(ReprTest.max_display_count + 1) self.assertTrue("Showing only the first" in psdf._repr_html_()) with option_context("display.max_rows", None): psdf = ps.range(ReprTest.max_display_count + 1) - self.assertEqual(psdf._repr_html_(), psdf.to_pandas()._repr_html_()) + self.assertEqual(psdf._repr_html_(), psdf._to_pandas()._repr_html_()) def test_repr_float_index(self): psdf = ps.DataFrame( @@ -154,14 +154,14 @@ def test_repr_float_index(self): index=np.random.rand(ReprTest.max_display_count), ) self.assertTrue("Showing only the first" not in repr(psdf)) - self.assert_eq(repr(psdf), repr(psdf.to_pandas())) + self.assert_eq(repr(psdf), repr(psdf._to_pandas())) self.assertTrue("Showing only the first" not in repr(psdf.a)) - self.assert_eq(repr(psdf.a), repr(psdf.a.to_pandas())) + self.assert_eq(repr(psdf.a), repr(psdf.a._to_pandas())) self.assertTrue("Showing only the first" not in repr(psdf.index)) - self.assert_eq(repr(psdf.index), repr(psdf.index.to_pandas())) + self.assert_eq(repr(psdf.index), repr(psdf.index._to_pandas())) self.assertTrue("Showing only the first" not in psdf._repr_html_()) - self.assertEqual(psdf._repr_html_(), psdf.to_pandas()._repr_html_()) + self.assertEqual(psdf._repr_html_(), psdf._to_pandas()._repr_html_()) psdf = ps.DataFrame( {"a": np.random.rand(ReprTest.max_display_count + 1)}, diff --git a/python/pyspark/pandas/tests/test_series.py b/python/pyspark/pandas/tests/test_series.py index db56e7e12da95..af6ecd6152b6e 100644 --- a/python/pyspark/pandas/tests/test_series.py +++ b/python/pyspark/pandas/tests/test_series.py @@ -982,7 +982,7 @@ def test_reset_index_with_default_index_types(self): with ps.option_context("compute.default_index_type", "distributed"): # the index is different. self.assert_eq( - psser.reset_index().to_pandas().reset_index(drop=True), pser.reset_index() + psser.reset_index()._to_pandas().reset_index(drop=True), pser.reset_index() ) def test_index_to_series_reset_index(self): @@ -2866,7 +2866,7 @@ def test_factorize(self): psser = ps.Series([1, 2, np.nan, 4, 5]) # Arrow takes np.nan as null psser.loc[3] = np.nan # Spark takes np.nan as NaN kcodes, kuniques = psser.factorize(na_sentinel=None) - pcodes, puniques = psser.to_pandas().factorize(sort=True, na_sentinel=None) + pcodes, puniques = psser._to_pandas().factorize(sort=True, na_sentinel=None) self.assert_eq(pcodes.tolist(), kcodes.to_list()) self.assert_eq(puniques, kuniques) diff --git a/python/pyspark/pandas/tests/test_series_datetime.py b/python/pyspark/pandas/tests/test_series_datetime.py index d837c34fc7439..1fe078e97261f 100644 --- a/python/pyspark/pandas/tests/test_series_datetime.py +++ b/python/pyspark/pandas/tests/test_series_datetime.py @@ -52,17 +52,17 @@ def test_timestamp_subtraction(self): pdf = self.pdf1 psdf = ps.from_pandas(pdf) - actual = (psdf["end_date"] - psdf["start_date"] - 1).to_pandas() + actual = (psdf["end_date"] - psdf["start_date"] - 1)._to_pandas() expected = (pdf["end_date"] - pdf["start_date"]) // np.timedelta64(1, "s") - 1 self.assert_eq(actual, expected) - actual = (psdf["end_date"] - pd.Timestamp("2012-1-1 12:45:31") - 1).to_pandas() + actual = (psdf["end_date"] - pd.Timestamp("2012-1-1 12:45:31") - 1)._to_pandas() expected = (pdf["end_date"] - pd.Timestamp("2012-1-1 12:45:31")) // np.timedelta64( 1, "s" ) - 1 self.assert_eq(actual, expected) - actual = (pd.Timestamp("2013-3-11 21:45:00") - psdf["start_date"] - 1).to_pandas() + actual = (pd.Timestamp("2013-3-11 21:45:00") - psdf["start_date"] - 1)._to_pandas() expected = (pd.Timestamp("2013-3-11 21:45:00") - pdf["start_date"]) // np.timedelta64( 1, "s" ) - 1 From 43a6b932759865c45ccf36f3e9cf6898c1b762da Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 30 Sep 2022 09:23:08 +0300 Subject: [PATCH 6/8] [SPARK-40540][SQL] Migrate compilation errors onto error classes: _LEGACY_ERROR_TEMP_1200-1299 ### What changes were proposed in this pull request? In the PR, I propose to migrate 100 compilation errors onto temporary error classes with the prefix `_LEGACY_ERROR_TEMP_12xx`. The error message will not include the error classes, so, in this way we will preserve the existing behaviour. ### Why are the changes needed? The migration on temporary error classes allows to gather statistics about errors and detect most popular error classes. After that we could prioritise the work on migration. The new error class name prefix `_LEGACY_ERROR_TEMP_` proposed here kind of marks the error as developer-facing, not user-facing. Developers can still get the error class programmatically via the `SparkThrowable` interface, so that they can build error infra with it. End users won't see the error class in the message. This allows us to do the error migration very quickly, and we can refine the error classes and mark them as user-facing later (naming them properly, adding tests, etc.). ### Does this PR introduce _any_ user-facing change? No. The error messages should be almost the same by default. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "core/testOnly *SparkThrowableSuite" $ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite" $ build/sbt "test:testOnly *SQLQuerySuite" ``` Closes #38027 from MaxGekk/legacy-error-temp-compliation-1200. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../main/resources/error/error-classes.json | 530 +++++++++++++++- .../analysis/NoSuchItemException.scala | 10 +- .../sql/errors/QueryCompilationErrors.scala | 598 +++++++++++------- .../encoders/EncoderResolutionSuite.scala | 8 +- .../sql-tests/results/ansi/interval.sql.out | 60 +- .../sql-tests/results/ansi/literals.sql.out | 24 +- .../sql-tests/results/change-column.sql.out | 10 +- .../sql-tests/results/charvarchar.sql.out | 19 +- .../sql-tests/results/cte-nested.sql.out | 64 +- .../sql-tests/results/describe.sql.out | 18 +- .../sql-tests/results/inline-table.sql.out | 15 +- .../sql-tests/results/interval.sql.out | 60 +- .../sql-tests/results/literals.sql.out | 24 +- .../results/postgreSQL/create_view.sql.out | 104 ++- .../results/postgreSQL/numeric.sql.out | 11 +- .../results/postgreSQL/strings.sql.out | 64 +- .../results/postgreSQL/window_part3.sql.out | 15 +- .../sql-tests/results/show-tables.sql.out | 26 +- .../results/udf/udf-inline-table.sql.out | 15 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 13 +- .../sql/execution/command/DDLSuite.scala | 30 +- .../v1/AlterTableSetLocationSuite.scala | 14 +- .../command/v1/AlterTableSetSerdeSuite.scala | 76 +-- 23 files changed, 1453 insertions(+), 355 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 48e90cc617d50..3fa7f7e9e4b73 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -2077,7 +2077,7 @@ }, "_LEGACY_ERROR_TEMP_1169" : { "message" : [ - "Requested partitioning does not match the table $tableName:", + "Requested partitioning does not match the table :", "Requested partitions: ", "Table partitions: " ] @@ -2130,7 +2130,7 @@ }, "_LEGACY_ERROR_TEMP_1179" : { "message" : [ - "Table-valued function $name with alternatives: ", + "Table-valued function with alternatives: ", "cannot be applied to ():
." ] }, @@ -2234,5 +2234,531 @@ "message" : [ "Invalid bound function ': there are arguments but parameters returned from 'inputTypes()'." ] + }, + "_LEGACY_ERROR_TEMP_1200" : { + "message" : [ + " is not supported for v2 tables." + ] + }, + "_LEGACY_ERROR_TEMP_1201" : { + "message" : [ + "Cannot resolve column name \"\" among ()." + ] + }, + "_LEGACY_ERROR_TEMP_1202" : { + "message" : [ + "Cannot write to '', too many data columns:", + "Table columns: ", + "Data columns: " + ] + }, + "_LEGACY_ERROR_TEMP_1203" : { + "message" : [ + "Cannot write to '', not enough data columns:", + "Table columns: ", + "Data columns: " + ] + }, + "_LEGACY_ERROR_TEMP_1204" : { + "message" : [ + "Cannot write incompatible data to table '':", + "- " + ] + }, + "_LEGACY_ERROR_TEMP_1205" : { + "message" : [ + "Expected only partition pruning predicates: ." + ] + }, + "_LEGACY_ERROR_TEMP_1206" : { + "message" : [ + " column is not defined in table , defined table columns are: ." + ] + }, + "_LEGACY_ERROR_TEMP_1207" : { + "message" : [ + "The duration and time inputs to window must be an integer, long or string literal." + ] + }, + "_LEGACY_ERROR_TEMP_1208" : { + "message" : [ + "No such struct field in ." + ] + }, + "_LEGACY_ERROR_TEMP_1209" : { + "message" : [ + "Ambiguous reference to fields ." + ] + }, + "_LEGACY_ERROR_TEMP_1210" : { + "message" : [ + "The second argument in should be a boolean literal." + ] + }, + "_LEGACY_ERROR_TEMP_1211" : { + "message" : [ + "Detected implicit cartesian product for join between logical plans", + "", + "and", + "rightPlan", + "Join condition is missing or trivial.", + "Either: use the CROSS JOIN syntax to allow cartesian products between these relations, or: enable implicit cartesian products by setting the configuration variable spark.sql.crossJoin.enabled=true." + ] + }, + "_LEGACY_ERROR_TEMP_1212" : { + "message" : [ + "Found conflicting attributes in the condition joining outer plan:", + "", + "and subplan:", + "" + ] + }, + "_LEGACY_ERROR_TEMP_1213" : { + "message" : [ + "Window expression is empty in ." + ] + }, + "_LEGACY_ERROR_TEMP_1214" : { + "message" : [ + "Found different window function type in ." + ] + }, + "_LEGACY_ERROR_TEMP_1215" : { + "message" : [ + "char/varchar type can only be used in the table schema. You can set to true, so that Spark treat them as string type as same as Spark 3.0 and earlier." + ] + }, + "_LEGACY_ERROR_TEMP_1216" : { + "message" : [ + "The pattern '' is invalid, ." + ] + }, + "_LEGACY_ERROR_TEMP_1217" : { + "message" : [ + " already exists." + ] + }, + "_LEGACY_ERROR_TEMP_1218" : { + "message" : [ + " should be converted to HadoopFsRelation." + ] + }, + "_LEGACY_ERROR_TEMP_1219" : { + "message" : [ + "Hive metastore does not support altering database location" + ] + }, + "_LEGACY_ERROR_TEMP_1220" : { + "message" : [ + "Hive is not supported." + ] + }, + "_LEGACY_ERROR_TEMP_1221" : { + "message" : [ + "Hive 0.12 doesn't support creating permanent functions. Please use Hive 0.13 or higher." + ] + }, + "_LEGACY_ERROR_TEMP_1222" : { + "message" : [ + "Unknown resource type: ." + ] + }, + "_LEGACY_ERROR_TEMP_1223" : { + "message" : [ + "Invalid field id '' in day-time interval. Supported interval fields: ." + ] + }, + "_LEGACY_ERROR_TEMP_1224" : { + "message" : [ + "'interval to ' is invalid." + ] + }, + "_LEGACY_ERROR_TEMP_1225" : { + "message" : [ + "Invalid field id '' in year-month interval. Supported interval fields: ." + ] + }, + "_LEGACY_ERROR_TEMP_1226" : { + "message" : [ + "The SQL config '' was removed in the version . " + ] + }, + "_LEGACY_ERROR_TEMP_1227" : { + "message" : [ + "", + "Failed fallback parsing: " + ] + }, + "_LEGACY_ERROR_TEMP_1228" : { + "message" : [ + "Decimal scale () cannot be greater than precision ()." + ] + }, + "_LEGACY_ERROR_TEMP_1229" : { + "message" : [ + " can only support precision up to ." + ] + }, + "_LEGACY_ERROR_TEMP_1230" : { + "message" : [ + "Negative scale is not allowed: . You can use =true to enable legacy mode to allow it." + ] + }, + "_LEGACY_ERROR_TEMP_1231" : { + "message" : [ + " is not a valid partition column in table ." + ] + }, + "_LEGACY_ERROR_TEMP_1232" : { + "message" : [ + "Partition spec is invalid. The spec () must match the partition spec () defined in table ''." + ] + }, + "_LEGACY_ERROR_TEMP_1233" : { + "message" : [ + "Found duplicate column(s) : ." + ] + }, + "_LEGACY_ERROR_TEMP_1234" : { + "message" : [ + "Temporary view is not cached for analyzing columns." + ] + }, + "_LEGACY_ERROR_TEMP_1235" : { + "message" : [ + "Column in table is of type , and Spark does not support statistics collection on this column type." + ] + }, + "_LEGACY_ERROR_TEMP_1236" : { + "message" : [ + "ANALYZE TABLE is not supported on views." + ] + }, + "_LEGACY_ERROR_TEMP_1237" : { + "message" : [ + "The list of partition columns with values in partition specification for table '' in database '' is not a prefix of the list of partition columns defined in the table schema. Expected a prefix of [], but got []." + ] + }, + "_LEGACY_ERROR_TEMP_1238" : { + "message" : [ + "." + ] + }, + "_LEGACY_ERROR_TEMP_1239" : { + "message" : [ + "Analyzing column statistics is not supported for column of data type: ." + ] + }, + "_LEGACY_ERROR_TEMP_1240" : { + "message" : [ + "Table
already exists." + ] + }, + "_LEGACY_ERROR_TEMP_1241" : { + "message" : [ + "CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory . To allow overwriting the existing non-empty directory, set '' to true." + ] + }, + "_LEGACY_ERROR_TEMP_1242" : { + "message" : [ + "Undefined function: . This function is neither a built-in/temporary function, nor a persistent function that is qualified as ." + ] + }, + "_LEGACY_ERROR_TEMP_1243" : { + "message" : [ + "Undefined function: " + ] + }, + "_LEGACY_ERROR_TEMP_1244" : { + "message" : [ + "Attempted to unset non-existent property '' in table '
'." + ] + }, + "_LEGACY_ERROR_TEMP_1245" : { + "message" : [ + "ALTER TABLE CHANGE COLUMN is not supported for changing column '' with type '' to '' with type ''." + ] + }, + "_LEGACY_ERROR_TEMP_1246" : { + "message" : [ + "Can't find column `` given table data columns ." + ] + }, + "_LEGACY_ERROR_TEMP_1247" : { + "message" : [ + "Operation not allowed: ALTER TABLE SET [SERDE | SERDEPROPERTIES] for a specific partition is not supported for tables created with the datasource API." + ] + }, + "_LEGACY_ERROR_TEMP_1248" : { + "message" : [ + "Operation not allowed: ALTER TABLE SET SERDE is not supported for tables created with the datasource API." + ] + }, + "_LEGACY_ERROR_TEMP_1249" : { + "message" : [ + "Operation not allowed: only works on partitioned tables: ." + ] + }, + "_LEGACY_ERROR_TEMP_1250" : { + "message" : [ + " is not allowed on since filesource partition management is disabled (spark.sql.hive.manageFilesourcePartitions = false)." + ] + }, + "_LEGACY_ERROR_TEMP_1251" : { + "message" : [ + " is not allowed on since its partition metadata is not stored in the Hive metastore. To import this information into the metastore, run `msck repair table `." + ] + }, + "_LEGACY_ERROR_TEMP_1252" : { + "message" : [ + "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead." + ] + }, + "_LEGACY_ERROR_TEMP_1253" : { + "message" : [ + "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead." + ] + }, + "_LEGACY_ERROR_TEMP_1254" : { + "message" : [ + "Cannot overwrite a path that is also being read from." + ] + }, + "_LEGACY_ERROR_TEMP_1255" : { + "message" : [ + "Cannot drop built-in function ''." + ] + }, + "_LEGACY_ERROR_TEMP_1256" : { + "message" : [ + "Cannot refresh built-in function ." + ] + }, + "_LEGACY_ERROR_TEMP_1257" : { + "message" : [ + "Cannot refresh temporary function ." + ] + }, + "_LEGACY_ERROR_TEMP_1258" : { + "message" : [ + "." + ] + }, + "_LEGACY_ERROR_TEMP_1259" : { + "message" : [ + "ALTER ADD COLUMNS does not support views. You must drop and re-create the views for adding the new columns. Views:
." + ] + }, + "_LEGACY_ERROR_TEMP_1260" : { + "message" : [ + "ALTER ADD COLUMNS does not support datasource table with type . You must drop and re-create the table for adding the new columns. Tables:
." + ] + }, + "_LEGACY_ERROR_TEMP_1261" : { + "message" : [ + "LOAD DATA is not supported for datasource tables: ." + ] + }, + "_LEGACY_ERROR_TEMP_1262" : { + "message" : [ + "LOAD DATA target table is partitioned, but no partition spec is provided." + ] + }, + "_LEGACY_ERROR_TEMP_1263" : { + "message" : [ + "LOAD DATA target table is partitioned, but number of columns in provided partition spec () do not match number of partitioned columns in table ()." + ] + }, + "_LEGACY_ERROR_TEMP_1264" : { + "message" : [ + "LOAD DATA target table is not partitioned, but a partition spec was provided." + ] + }, + "_LEGACY_ERROR_TEMP_1265" : { + "message" : [ + "LOAD DATA input path does not exist: ." + ] + }, + "_LEGACY_ERROR_TEMP_1266" : { + "message" : [ + "Operation not allowed: TRUNCATE TABLE on external tables: ." + ] + }, + "_LEGACY_ERROR_TEMP_1267" : { + "message" : [ + "Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported for tables that are not partitioned: ." + ] + }, + "_LEGACY_ERROR_TEMP_1268" : { + "message" : [ + "Failed to truncate table when removing data of the path: ." + ] + }, + "_LEGACY_ERROR_TEMP_1269" : { + "message" : [ + "SHOW PARTITIONS is not allowed on a table that is not partitioned: ." + ] + }, + "_LEGACY_ERROR_TEMP_1270" : { + "message" : [ + "SHOW CREATE TABLE is not supported on a temporary view:
" + ] + }, + "_LEGACY_ERROR_TEMP_1271" : { + "message" : [ + "Failed to execute SHOW CREATE TABLE against table
, which is created by Hive and uses the following unsupported feature(s)", + "", + "Please use `SHOW CREATE TABLE
AS SERDE` to show Hive DDL instead." + ] + }, + "_LEGACY_ERROR_TEMP_1272" : { + "message" : [ + "SHOW CREATE TABLE doesn't support transactional Hive table. Please use `SHOW CREATE TABLE
AS SERDE` to show Hive DDL instead." + ] + }, + "_LEGACY_ERROR_TEMP_1273" : { + "message" : [ + "Failed to execute SHOW CREATE TABLE against table
, which is created by Hive and uses the following unsupported serde configuration", + "", + "Please use `SHOW CREATE TABLE
AS SERDE` to show Hive DDL instead." + ] + }, + "_LEGACY_ERROR_TEMP_1274" : { + "message" : [ + "
is a Spark data source table. Use `SHOW CREATE TABLE` without `AS SERDE` instead." + ] + }, + "_LEGACY_ERROR_TEMP_1275" : { + "message" : [ + "Failed to execute SHOW CREATE TABLE against table/view
, which is created by Hive and uses the following unsupported feature(s)", + "" + ] + }, + "_LEGACY_ERROR_TEMP_1276" : { + "message" : [ + "The logical plan that represents the view is not analyzed." + ] + }, + "_LEGACY_ERROR_TEMP_1277" : { + "message" : [ + "The number of columns produced by the SELECT clause (num: ``) does not match the number of column names specified by CREATE VIEW (num: ``)." + ] + }, + "_LEGACY_ERROR_TEMP_1278" : { + "message" : [ + " is not a view." + ] + }, + "_LEGACY_ERROR_TEMP_1279" : { + "message" : [ + "View already exists. If you want to update the view definition, please use ALTER VIEW AS or CREATE OR REPLACE VIEW AS." + ] + }, + "_LEGACY_ERROR_TEMP_1280" : { + "message" : [ + "It is not allowed to create a persisted view from the Dataset API." + ] + }, + "_LEGACY_ERROR_TEMP_1281" : { + "message" : [ + "Recursive view detected (cycle: )." + ] + }, + "_LEGACY_ERROR_TEMP_1282" : { + "message" : [ + "Not allowed to create a permanent view without explicitly assigning an alias for expression ." + ] + }, + "_LEGACY_ERROR_TEMP_1283" : { + "message" : [ + "Not allowed to create a permanent view by referencing a temporary view . Please create a temp view instead by CREATE TEMP VIEW." + ] + }, + "_LEGACY_ERROR_TEMP_1284" : { + "message" : [ + "Not allowed to create a permanent view by referencing a temporary function ``." + ] + }, + "_LEGACY_ERROR_TEMP_1285" : { + "message" : [ + "Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the", + "referenced columns only include the internal corrupt record column", + "(named _corrupt_record by default). For example:", + "spark.read.schema(schema).csv(file).filter($\"_corrupt_record\".isNotNull).count()", + "and spark.read.schema(schema).csv(file).select(\"_corrupt_record\").show().", + "Instead, you can cache or save the parsed results and then send the same query.", + "For example, val df = spark.read.schema(schema).csv(file).cache() and then", + "df.filter($\"_corrupt_record\".isNotNull).count()." + ] + }, + "_LEGACY_ERROR_TEMP_1286" : { + "message" : [ + "User-defined partition column not found in the JDBC relation: ." + ] + }, + "_LEGACY_ERROR_TEMP_1287" : { + "message" : [ + "Partition column type should be , , or , but found." + ] + }, + "_LEGACY_ERROR_TEMP_1288" : { + "message" : [ + "Table or view '' already exists. SaveMode: ErrorIfExists." + ] + }, + "_LEGACY_ERROR_TEMP_1289" : { + "message" : [ + "Column name \"\" contains invalid character(s). Please use alias to rename it." + ] + }, + "_LEGACY_ERROR_TEMP_1290" : { + "message" : [ + "Text data source supports only a single column, and you have columns." + ] + }, + "_LEGACY_ERROR_TEMP_1291" : { + "message" : [ + "Can't find required partition column in partition schema ." + ] + }, + "_LEGACY_ERROR_TEMP_1292" : { + "message" : [ + "Temporary view '' should not have specified a database." + ] + }, + "_LEGACY_ERROR_TEMP_1293" : { + "message" : [ + "Hive data source can only be used with tables, you can't use it with CREATE TEMP VIEW USING." + ] + }, + "_LEGACY_ERROR_TEMP_1294" : { + "message" : [ + "The timestamp provided for the '' option is invalid. The expected format is 'YYYY-MM-DDTHH:mm:ss', but the provided timestamp: ." + ] + }, + "_LEGACY_ERROR_TEMP_1295" : { + "message" : [ + "Set a host to read from with option(\"host\", ...)." + ] + }, + "_LEGACY_ERROR_TEMP_1296" : { + "message" : [ + "Set a port to read from with option(\"port\", ...)." + ] + }, + "_LEGACY_ERROR_TEMP_1297" : { + "message" : [ + "IncludeTimestamp must be set to either \"true\" or \"false\"." + ] + }, + "_LEGACY_ERROR_TEMP_1298" : { + "message" : [ + "checkpointLocation must be specified either through option(\"checkpointLocation\", ...) or SparkSession.conf.set(\"\", ...)." + ] + }, + "_LEGACY_ERROR_TEMP_1299" : { + "message" : [ + "This query does not support recovering from checkpoint location. Delete to start over." + ] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index bf990afad6d30..6b4353acc96bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -66,7 +66,10 @@ case class NoSuchTableException( case class NoSuchPartitionException( override val message: String) - extends AnalysisException(message) { + extends AnalysisException( + message, + errorClass = Some("_LEGACY_ERROR_TEMP_1238"), + messageParameters = Map("msg" -> message)) { def this(db: String, table: String, spec: TablePartitionSpec) = { this(s"Partition not found in table '$table' database '$db':\n" + spec.mkString("\n")) @@ -83,7 +86,10 @@ case class NoSuchPermanentFunctionException(db: String, func: String) extends AnalysisException(s"Function '$func' not found in database '$db'") case class NoSuchFunctionException(override val message: String) - extends AnalysisException(message) { + extends AnalysisException( + message, + errorClass = Some("_LEGACY_ERROR_TEMP_1258"), + messageParameters = Map("msg" -> message)) { def this(db: String, func: String) = { this(s"Undefined function: '$func'. " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index f1da38f1734ec..f62620fcd7a5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1979,43 +1979,54 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def ambiguousRelationAliasNameInNestedCTEError(name: String): Throwable = { - new AnalysisException(s"Name $name is ambiguous in nested CTE. " + - s"Please set ${LEGACY_CTE_PRECEDENCE_POLICY.key} to CORRECTED so that name " + - "defined in inner CTE takes precedence. If set it to LEGACY, outer CTE " + - "definitions will take precedence. See more details in SPARK-28228.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1200", + messageParameters = Map( + "name" -> name, + "config" -> LEGACY_CTE_PRECEDENCE_POLICY.key)) } def commandUnsupportedInV2TableError(name: String): Throwable = { - new AnalysisException(s"$name is not supported for v2 tables.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1200", + messageParameters = Map("name" -> name)) } def cannotResolveColumnNameAmongAttributesError( colName: String, fieldNames: String): Throwable = { - new AnalysisException(s"""Cannot resolve column name "$colName" among ($fieldNames)""") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1201", + messageParameters = Map( + "colName" -> colName, + "fieldNames" -> fieldNames)) } def cannotWriteTooManyColumnsToTableError( tableName: String, expected: Seq[Attribute], query: LogicalPlan): Throwable = { new AnalysisException( - s""" - |Cannot write to '$tableName', too many data columns: - |Table columns: ${expected.map(c => s"'${c.name}'").mkString(", ")} - |Data columns: ${query.output.map(c => s"'${c.name}'").mkString(", ")} - """.stripMargin) + errorClass = "_LEGACY_ERROR_TEMP_1202", + messageParameters = Map( + "tableName" -> tableName, + "tableColumns" -> expected.map(c => s"'${c.name}'").mkString(", "), + "dataColumns" -> query.output.map(c => s"'${c.name}'").mkString(", "))) } def cannotWriteNotEnoughColumnsToTableError( tableName: String, expected: Seq[Attribute], query: LogicalPlan): Throwable = { new AnalysisException( - s"""Cannot write to '$tableName', not enough data columns: - |Table columns: ${expected.map(c => s"'${c.name}'").mkString(", ")} - |Data columns: ${query.output.map(c => s"'${c.name}'").mkString(", ")}""" - .stripMargin) + errorClass = "_LEGACY_ERROR_TEMP_1203", + messageParameters = Map( + "tableName" -> tableName, + "tableColumns" -> expected.map(c => s"'${c.name}'").mkString(", "), + "dataColumns" -> query.output.map(c => s"'${c.name}'").mkString(", "))) } def cannotWriteIncompatibleDataToTableError(tableName: String, errors: Seq[String]): Throwable = { new AnalysisException( - s"Cannot write incompatible data to table '$tableName':\n- ${errors.mkString("\n- ")}") + errorClass = "_LEGACY_ERROR_TEMP_1204", + messageParameters = Map( + "tableName" -> tableName, + "errors" -> errors.mkString("\n- "))) } def secondArgumentOfFunctionIsNotIntegerError( @@ -2030,46 +2041,57 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def nonPartitionPruningPredicatesNotExpectedError( nonPartitionPruningPredicates: Seq[Expression]): Throwable = { new AnalysisException( - s"Expected only partition pruning predicates: $nonPartitionPruningPredicates") + errorClass = "_LEGACY_ERROR_TEMP_1205", + messageParameters = Map( + "nonPartitionPruningPredicates" -> nonPartitionPruningPredicates.toString())) } def columnNotDefinedInTableError( colType: String, colName: String, tableName: String, tableCols: Seq[String]): Throwable = { - new AnalysisException(s"$colType column $colName is not defined in table $tableName, " + - s"defined table columns are: ${tableCols.mkString(", ")}") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1206", + messageParameters = Map( + "colType" -> colType, + "colName" -> colName, + "tableName" -> tableName, + "tableCols" -> tableCols.mkString(", "))) } def invalidLiteralForWindowDurationError(): Throwable = { - new AnalysisException("The duration and time inputs to window must be " + - "an integer, long or string literal.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1207", + messageParameters = Map.empty) } def noSuchStructFieldInGivenFieldsError( fieldName: String, fields: Array[StructField]): Throwable = { new AnalysisException( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") + errorClass = "_LEGACY_ERROR_TEMP_1208", + messageParameters = Map( + "fieldName" -> fieldName, + "fields" -> fields.map(_.name).mkString(", "))) } def ambiguousReferenceToFieldsError(fields: String): Throwable = { - new AnalysisException(s"Ambiguous reference to fields $fields") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1209", + messageParameters = Map("fields" -> fields)) } def secondArgumentInFunctionIsNotBooleanLiteralError(funcName: String): Throwable = { - new AnalysisException(s"The second argument in $funcName should be a boolean literal.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1210", + messageParameters = Map("funcName" -> funcName)) } def joinConditionMissingOrTrivialError( join: Join, left: LogicalPlan, right: LogicalPlan): Throwable = { new AnalysisException( - s"""Detected implicit cartesian product for ${join.joinType.sql} join between logical plans - |${left.treeString(false).trim} - |and - |${right.treeString(false).trim} - |Join condition is missing or trivial. - |Either: use the CROSS JOIN syntax to allow cartesian products between these - |relations, or: enable implicit cartesian products by setting the configuration - |variable spark.sql.crossJoin.enabled=true""" - .stripMargin) + errorClass = "_LEGACY_ERROR_TEMP_1211", + messageParameters = Map( + "joinType" -> join.joinType.sql, + "leftPlan" -> left.treeString(false).trim, + "rightPlan" -> right.treeString(false).trim)) } def usePythonUDFInJoinConditionUnsupportedError(joinType: JoinType): Throwable = { @@ -2081,73 +2103,101 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def conflictingAttributesInJoinConditionError( conflictingAttrs: AttributeSet, outerPlan: LogicalPlan, subplan: LogicalPlan): Throwable = { - new AnalysisException("Found conflicting attributes " + - s"${conflictingAttrs.mkString(",")} in the condition joining outer plan:\n " + - s"$outerPlan\nand subplan:\n $subplan") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1212", + messageParameters = Map( + "conflictingAttrs" -> conflictingAttrs.mkString(","), + "outerPlan" -> outerPlan.toString, + "subplan" -> subplan.toString)) } def emptyWindowExpressionError(expr: Window): Throwable = { - new AnalysisException(s"Window expression is empty in $expr") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1213", + messageParameters = Map("expr" -> expr.toString)) } def foundDifferentWindowFunctionTypeError(windowExpressions: Seq[NamedExpression]): Throwable = { new AnalysisException( - s"Found different window function type in $windowExpressions") + errorClass = "_LEGACY_ERROR_TEMP_1214", + messageParameters = Map("windowExpressions" -> windowExpressions.toString())) } def charOrVarcharTypeAsStringUnsupportedError(): Throwable = { - new AnalysisException("char/varchar type can only be used in the table schema. " + - s"You can set ${SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key} to true, so that Spark" + - s" treat them as string type as same as Spark 3.0 and earlier") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1215", + messageParameters = Map("config" -> SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key)) } def invalidPatternError(pattern: String, message: String): Throwable = { new AnalysisException( - s"the pattern '$pattern' is invalid, $message") + errorClass = "_LEGACY_ERROR_TEMP_1216", + messageParameters = Map("pattern" -> pattern, "message" -> message)) } def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { - new AnalysisException(s"$tableIdentifier already exists.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1217", + messageParameters = Map("tableIdentifier" -> tableIdentifier.toString)) } def tableIdentifierNotConvertedToHadoopFsRelationError( tableIdentifier: TableIdentifier): Throwable = { - new AnalysisException(s"$tableIdentifier should be converted to HadoopFsRelation.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1218", + messageParameters = Map("tableIdentifier" -> tableIdentifier.toString)) } def alterDatabaseLocationUnsupportedError(): Throwable = { - new AnalysisException("Hive metastore does not support altering database location") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1219", + messageParameters = Map.empty) } def hiveTableTypeUnsupportedError(tableType: String): Throwable = { - new AnalysisException(s"Hive $tableType is not supported.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1220", + messageParameters = Map("tableType" -> tableType)) } def hiveCreatePermanentFunctionsUnsupportedError(): Throwable = { - new AnalysisException("Hive 0.12 doesn't support creating permanent functions. " + - "Please use Hive 0.13 or higher.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1221", + messageParameters = Map.empty) } def unknownHiveResourceTypeError(resourceType: String): Throwable = { - new AnalysisException(s"Unknown resource type: $resourceType") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1222", + messageParameters = Map("resourceType" -> resourceType)) } def invalidDayTimeField(field: Byte): Throwable = { val supportedIds = DayTimeIntervalType.dayTimeFields .map(i => s"$i (${DayTimeIntervalType.fieldToString(i)})") - new AnalysisException(s"Invalid field id '$field' in day-time interval. " + - s"Supported interval fields: ${supportedIds.mkString(", ")}.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1223", + messageParameters = Map( + "field" -> field.toString, + "supportedIds" -> supportedIds.mkString(", "))) } def invalidDayTimeIntervalType(startFieldName: String, endFieldName: String): Throwable = { - new AnalysisException(s"'interval $startFieldName to $endFieldName' is invalid.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1224", + messageParameters = Map( + "startFieldName" -> startFieldName, + "endFieldName" -> endFieldName)) } def invalidYearMonthField(field: Byte): Throwable = { val supportedIds = YearMonthIntervalType.yearMonthFields .map(i => s"$i (${YearMonthIntervalType.fieldToString(i)})") - new AnalysisException(s"Invalid field id '$field' in year-month interval. " + - s"Supported interval fields: ${supportedIds.mkString(", ")}.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1225", + messageParameters = Map( + "field" -> field.toString, + "supportedIds" -> supportedIds.mkString(", "))) } def configRemovedInVersionError( @@ -2155,31 +2205,50 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { version: String, comment: String): Throwable = { new AnalysisException( - s"The SQL config '$configName' was removed in the version $version. $comment") + errorClass = "_LEGACY_ERROR_TEMP_1226", + messageParameters = Map( + "configName" -> configName, + "version" -> version, + "comment" -> comment)) } def failedFallbackParsingError(msg: String, e1: Throwable, e2: Throwable): Throwable = { - new AnalysisException(s"$msg${e1.getMessage}\nFailed fallback parsing: ${e2.getMessage}", + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1227", + messageParameters = Map("msg" -> msg, "e1" -> e1.getMessage, "e2" -> e2.getMessage), cause = Some(e1.getCause)) } def decimalCannotGreaterThanPrecisionError(scale: Int, precision: Int): Throwable = { - new AnalysisException(s"Decimal scale ($scale) cannot be greater than precision ($precision).") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1228", + messageParameters = Map( + "scale" -> scale.toString, + "precision" -> precision.toString)) } def decimalOnlySupportPrecisionUptoError(decimalType: String, precision: Int): Throwable = { - new AnalysisException(s"$decimalType can only support precision up to $precision") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1229", + messageParameters = Map( + "decimalType" -> decimalType, + "precision" -> precision.toString)) } def negativeScaleNotAllowedError(scale: Int): Throwable = { new AnalysisException( - s"""|Negative scale is not allowed: $scale. - |You can use ${LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key}=true - |to enable legacy mode to allow it.""".stripMargin.replaceAll("\n", " ")) + errorClass = "_LEGACY_ERROR_TEMP_1230", + messageParameters = Map( + "scale" -> scale.toString, + "config" -> LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key)) } def invalidPartitionColumnKeyInTableError(key: String, tblName: String): Throwable = { - new AnalysisException(s"$key is not a valid partition column in table $tblName.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1231", + messageParameters = Map( + "key" -> key, + "tblName" -> tblName)) } def invalidPartitionSpecError( @@ -2187,14 +2256,19 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { partitionColumnNames: Seq[String], tableName: String): Throwable = { new AnalysisException( - s"""|Partition spec is invalid. The spec ($specKeys) must match - |the partition spec (${partitionColumnNames.mkString(", ")}) defined in - |table '$tableName'""".stripMargin.replaceAll("\n", " ")) + errorClass = "_LEGACY_ERROR_TEMP_1232", + messageParameters = Map( + "specKeys" -> specKeys, + "partitionColumnNames" -> partitionColumnNames.mkString(", "), + "tableName" -> tableName)) } def foundDuplicateColumnError(colType: String, duplicateCol: Seq[String]): Throwable = { new AnalysisException( - s"Found duplicate column(s) $colType: ${duplicateCol.sorted.mkString(", ")}") + errorClass = "_LEGACY_ERROR_TEMP_1233", + messageParameters = Map( + "colType" -> colType, + "duplicateCol" -> duplicateCol.sorted.mkString(", "))) } def noSuchTableError(db: String, table: String): Throwable = { @@ -2202,19 +2276,27 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tempViewNotCachedForAnalyzingColumnsError(tableIdent: TableIdentifier): Throwable = { - new AnalysisException(s"Temporary view $tableIdent is not cached for analyzing columns.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1234", + messageParameters = Map("tableIdent" -> tableIdent.toString)) } def columnTypeNotSupportStatisticsCollectionError( name: String, tableIdent: TableIdentifier, dataType: DataType): Throwable = { - new AnalysisException(s"Column $name in table $tableIdent is of type $dataType, " + - "and Spark does not support statistics collection on this column type.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1235", + messageParameters = Map( + "name" -> name, + "tableIdent" -> tableIdent.toString, + "dataType" -> dataType.toString)) } def analyzeTableNotSupportedOnViewsError(): Throwable = { - new AnalysisException("ANALYZE TABLE is not supported on views.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1236", + messageParameters = Map.empty) } def unexpectedPartitionColumnPrefixError( @@ -2223,13 +2305,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { schemaColumns: String, specColumns: String): Throwable = { new AnalysisException( - s""" - |The list of partition columns with values - |in partition specification for table '${table}' - |in database '${database}' is not a prefix of the list of - |partition columns defined in the table schema. - |Expected a prefix of [${schemaColumns}], but got [${specColumns}]. - """.stripMargin.replaceAll("\n", " ")) + errorClass = "_LEGACY_ERROR_TEMP_1237", + messageParameters = Map( + "table" -> table, + "database" -> database, + "schemaColumns" -> schemaColumns, + "specColumns" -> specColumns)) } def noSuchPartitionError( @@ -2242,23 +2323,27 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def analyzingColumnStatisticsNotSupportedForColumnTypeError( name: String, dataType: DataType): Throwable = { - new AnalysisException("Analyzing column statistics is not supported for column " + - s"$name of data type: $dataType.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1239", + messageParameters = Map( + "name" -> name, + "dataType" -> dataType.toString)) } def tableAlreadyExistsError(table: String, guide: String = ""): Throwable = { - new AnalysisException(s"Table $table already exists." + guide) + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1240", + messageParameters = Map( + "table" -> table, + "guide" -> guide)) } def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String): Throwable = { new AnalysisException( - s"CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory " + - s"${tablePath} . To allow overwriting the existing non-empty directory, " + - s"set '${SQLConf.ALLOW_NON_EMPTY_LOCATION_IN_CTAS.key}' to true.") - } - - def tableOrViewNotFoundError(table: String): Throwable = { - new AnalysisException(s"Table or view not found: $table") + errorClass = "_LEGACY_ERROR_TEMP_1241", + messageParameters = Map( + "tablePath" -> tablePath, + "config" -> SQLConf.ALLOW_NON_EMPTY_LOCATION_IN_CTAS.key)) } def noSuchFunctionError( @@ -2266,50 +2351,76 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { t: TreeNode[_], fullName: Option[Seq[String]] = None): Throwable = { if (rawName.length == 1 && fullName.isDefined) { - new AnalysisException(s"Undefined function: ${rawName.head}. " + - "This function is neither a built-in/temporary function, nor a persistent " + - s"function that is qualified as ${fullName.get.quoted}.", - t.origin.line, t.origin.startPosition) + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1242", + messageParameters = Map( + "rawName" -> rawName.head, + "fullName" -> fullName.get.quoted + ), + origin = t.origin) } else { - new AnalysisException(s"Undefined function: ${rawName.quoted}", - t.origin.line, t.origin.startPosition) + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1243", + messageParameters = Map("rawName" -> rawName.quoted), + origin = t.origin) } } def unsetNonExistentPropertyError(property: String, table: TableIdentifier): Throwable = { - new AnalysisException(s"Attempted to unset non-existent property '$property' in table '$table'") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1244", + messageParameters = Map( + "property" -> property, + "table" -> table.toString)) } def alterTableChangeColumnNotSupportedForColumnTypeError( originColumn: StructField, newColumn: StructField): Throwable = { - new AnalysisException("ALTER TABLE CHANGE COLUMN is not supported for changing column " + - s"'${originColumn.name}' with type '${originColumn.dataType}' to " + - s"'${newColumn.name}' with type '${newColumn.dataType}'") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1245", + messageParameters = Map( + "originName" -> originColumn.name, + "originType" -> originColumn.dataType.toString, + "newName" -> newColumn.name, + "newType"-> newColumn.dataType.toString)) } def cannotFindColumnError(name: String, fieldNames: Array[String]): Throwable = { - new AnalysisException(s"Can't find column `$name` given table data columns " + - s"${fieldNames.mkString("[`", "`, `", "`]")}") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1246", + messageParameters = Map( + "name" -> name, + "fieldNames" -> fieldNames.mkString("[`", "`, `", "`]"))) + } def alterTableSetSerdeForSpecificPartitionNotSupportedError(): Throwable = { - new AnalysisException("Operation not allowed: ALTER TABLE SET " + - "[SERDE | SERDEPROPERTIES] for a specific partition is not supported " + - "for tables created with the datasource API") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1247", + messageParameters = Map.empty) } def alterTableSetSerdeNotSupportedError(): Throwable = { - new AnalysisException("Operation not allowed: ALTER TABLE SET SERDE is " + - "not supported for tables created with the datasource API") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1248", + messageParameters = Map.empty) } def cmdOnlyWorksOnPartitionedTablesError(cmd: String, tableIdentWithDB: String): Throwable = { new AnalysisException( - s"Operation not allowed: $cmd only works on partitioned tables: $tableIdentWithDB") + errorClass = "_LEGACY_ERROR_TEMP_1249", + messageParameters = Map( + "cmd" -> cmd, + "tableIdentWithDB" -> tableIdentWithDB)) } def cmdOnlyWorksOnTableWithLocationError(cmd: String, tableIdentWithDB: String): Throwable = { + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1249", + messageParameters = Map( + "cmd" -> cmd, + "tableIdentWithDB" -> tableIdentWithDB)) new AnalysisException(s"Operation not allowed: $cmd only works on table with " + s"location provided: $tableIdentWithDB") } @@ -2318,43 +2429,56 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { action: String, tableName: String): Throwable = { new AnalysisException( - s"$action is not allowed on $tableName since filesource partition management is " + - "disabled (spark.sql.hive.manageFilesourcePartitions = false).") + errorClass = "_LEGACY_ERROR_TEMP_1250", + messageParameters = Map( + "action" -> action, + "tableName" -> tableName)) } def actionNotAllowedOnTableSincePartitionMetadataNotStoredError( action: String, tableName: String): Throwable = { new AnalysisException( - s"$action is not allowed on $tableName since its partition metadata is not stored in " + - "the Hive metastore. To import this information into the metastore, run " + - s"`msck repair table $tableName`") + errorClass = "_LEGACY_ERROR_TEMP_1251", + messageParameters = Map( + "action" -> action, + "tableName" -> tableName)) } def cannotAlterViewWithAlterTableError(): Throwable = { new AnalysisException( - "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead") + errorClass = "_LEGACY_ERROR_TEMP_1252", + messageParameters = Map.empty) } def cannotAlterTableWithAlterViewError(): Throwable = { new AnalysisException( - "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead") + errorClass = "_LEGACY_ERROR_TEMP_1253", + messageParameters = Map.empty) } def cannotOverwritePathBeingReadFromError(): Throwable = { - new AnalysisException("Cannot overwrite a path that is also being read from.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1254", + messageParameters = Map.empty) } def cannotDropBuiltinFuncError(functionName: String): Throwable = { - new AnalysisException(s"Cannot drop built-in function '$functionName'") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1255", + messageParameters = Map("functionName" -> functionName)) } def cannotRefreshBuiltInFuncError(functionName: String): Throwable = { - new AnalysisException(s"Cannot refresh built-in function $functionName") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1256", + messageParameters = Map("functionName" -> functionName)) } def cannotRefreshTempFuncError(functionName: String): Throwable = { - new AnalysisException(s"Cannot refresh temporary function $functionName") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1257", + messageParameters = Map("functionName" -> functionName)) } def noSuchFunctionError(identifier: FunctionIdentifier): Throwable = { @@ -2363,29 +2487,30 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def alterAddColNotSupportViewError(table: TableIdentifier): Throwable = { new AnalysisException( - s""" - |ALTER ADD COLUMNS does not support views. - |You must drop and re-create the views for adding the new columns. Views: $table - """.stripMargin) + errorClass = "_LEGACY_ERROR_TEMP_1259", + messageParameters = Map("table" -> table.toString)) } def alterAddColNotSupportDatasourceTableError( tableType: Any, table: TableIdentifier): Throwable = { new AnalysisException( - s""" - |ALTER ADD COLUMNS does not support datasource table with type $tableType. - |You must drop and re-create the table for adding the new columns. Tables: $table - """.stripMargin) + errorClass = "_LEGACY_ERROR_TEMP_1260", + messageParameters = Map( + "tableType" -> tableType.toString, + "table" -> table.toString)) } def loadDataNotSupportedForDatasourceTablesError(tableIdentWithDB: String): Throwable = { - new AnalysisException(s"LOAD DATA is not supported for datasource tables: $tableIdentWithDB") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1261", + messageParameters = Map("tableIdentWithDB" -> tableIdentWithDB)) } def loadDataWithoutPartitionSpecProvidedError(tableIdentWithDB: String): Throwable = { - new AnalysisException(s"LOAD DATA target table $tableIdentWithDB is partitioned, " + - s"but no partition spec is provided") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1262", + messageParameters = Map("tableIdentWithDB" -> tableIdentWithDB)) } def loadDataPartitionSizeNotMatchNumPartitionColumnsError( @@ -2393,40 +2518,49 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { partitionSize: Int, targetTableSize: Int): Throwable = { new AnalysisException( - s""" - |LOAD DATA target table $tableIdentWithDB is partitioned, - |but number of columns in provided partition spec ($partitionSize) - |do not match number of partitioned columns in table ($targetTableSize) - """.stripMargin.replaceAll("\n", " ")) + errorClass = "_LEGACY_ERROR_TEMP_1263", + messageParameters = Map( + "partitionSize" -> partitionSize.toString, + "targetTableSize" -> targetTableSize.toString, + "tableIdentWithDB" -> tableIdentWithDB)) } def loadDataTargetTableNotPartitionedButPartitionSpecWasProvidedError( tableIdentWithDB: String): Throwable = { - new AnalysisException(s"LOAD DATA target table $tableIdentWithDB is not " + - s"partitioned, but a partition spec was provided.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1264", + messageParameters = Map("tableIdentWithDB" -> tableIdentWithDB)) } def loadDataInputPathNotExistError(path: String): Throwable = { - new AnalysisException(s"LOAD DATA input path does not exist: $path") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1265", + messageParameters = Map("path" -> path)) } def truncateTableOnExternalTablesError(tableIdentWithDB: String): Throwable = { new AnalysisException( - s"Operation not allowed: TRUNCATE TABLE on external tables: $tableIdentWithDB") + errorClass = "_LEGACY_ERROR_TEMP_1266", + messageParameters = Map("tableIdentWithDB" -> tableIdentWithDB)) } def truncateTablePartitionNotSupportedForNotPartitionedTablesError( tableIdentWithDB: String): Throwable = { - new AnalysisException(s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported" + - s" for tables that are not partitioned: $tableIdentWithDB") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1267", + messageParameters = Map("tableIdentWithDB" -> tableIdentWithDB)) } def failToTruncateTableWhenRemovingDataError( tableIdentWithDB: String, path: Path, e: Throwable): Throwable = { - new AnalysisException(s"Failed to truncate table $tableIdentWithDB when " + - s"removing data of the path: $path because of ${e.toString}", cause = Some(e)) + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1268", + messageParameters = Map( + "tableIdentWithDB" -> tableIdentWithDB, + "path" -> path.toString), + cause = Some(e)) } def descPartitionNotAllowedOnTempView(table: String): Throwable = { @@ -2449,202 +2583,232 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def showPartitionNotAllowedOnTableNotPartitionedError(tableIdentWithDB: String): Throwable = { new AnalysisException( - s"SHOW PARTITIONS is not allowed on a table that is not partitioned: $tableIdentWithDB") + errorClass = "_LEGACY_ERROR_TEMP_1269", + messageParameters = Map("tableIdentWithDB" -> tableIdentWithDB)) } def showCreateTableNotSupportedOnTempView(table: String): Throwable = { - new AnalysisException(s"SHOW CREATE TABLE is not supported on a temporary view: $table") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1270", + messageParameters = Map("table" -> table)) } def showCreateTableFailToExecuteUnsupportedFeatureError(table: CatalogTable): Throwable = { - new AnalysisException("Failed to execute SHOW CREATE TABLE against table " + - s"${table.identifier}, which is created by Hive and uses the " + - s"following unsupported feature(s)\n" + - table.unsupportedFeatures.map(" - " + _).mkString("\n") + ". " + - s"Please use `SHOW CREATE TABLE ${table.identifier} AS SERDE` to show Hive DDL instead.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1271", + messageParameters = Map( + "unsupportedFeatures" -> table.unsupportedFeatures.map(" - " + _).mkString("\n"), + "table" -> table.identifier.toString)) } def showCreateTableNotSupportTransactionalHiveTableError(table: CatalogTable): Throwable = { - new AnalysisException("SHOW CREATE TABLE doesn't support transactional Hive table. " + - s"Please use `SHOW CREATE TABLE ${table.identifier} AS SERDE` " + - "to show Hive DDL instead.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1272", + messageParameters = Map("table" -> table.identifier.toString)) } def showCreateTableFailToExecuteUnsupportedConfError( table: TableIdentifier, builder: mutable.StringBuilder): Throwable = { - new AnalysisException("Failed to execute SHOW CREATE TABLE against table " + - s"${table.identifier}, which is created by Hive and uses the " + - "following unsupported serde configuration\n" + - builder.toString() + "\n" + - s"Please use `SHOW CREATE TABLE ${table.identifier} AS SERDE` to show Hive DDL instead." - ) + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1273", + messageParameters = Map( + "table" -> table.identifier, + "configs" -> builder.toString())) } def showCreateTableAsSerdeNotAllowedOnSparkDataSourceTableError( table: TableIdentifier): Throwable = { new AnalysisException( - s"$table is a Spark data source table. Use `SHOW CREATE TABLE` without `AS SERDE` instead.") + errorClass = "_LEGACY_ERROR_TEMP_1274", + messageParameters = Map("table" -> table.toString)) } def showCreateTableOrViewFailToExecuteUnsupportedFeatureError( table: CatalogTable, features: Seq[String]): Throwable = { new AnalysisException( - s"Failed to execute SHOW CREATE TABLE against table/view ${table.identifier}, " + - "which is created by Hive and uses the following unsupported feature(s)\n" + - features.map(" - " + _).mkString("\n")) + errorClass = "_LEGACY_ERROR_TEMP_1275", + messageParameters = Map( + "table" -> table.identifier.toString, + "features" -> features.map(" - " + _).mkString("\n"))) } def logicalPlanForViewNotAnalyzedError(): Throwable = { - new AnalysisException("The logical plan that represents the view is not analyzed.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1276", + messageParameters = Map.empty) } def createViewNumColumnsMismatchUserSpecifiedColumnLengthError( analyzedPlanLength: Int, userSpecifiedColumnsLength: Int): Throwable = { - new AnalysisException(s"The number of columns produced by the SELECT clause " + - s"(num: `$analyzedPlanLength`) does not match the number of column names " + - s"specified by CREATE VIEW (num: `$userSpecifiedColumnsLength`).") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1277", + messageParameters = Map( + "analyzedPlanLength" -> analyzedPlanLength.toString, + "userSpecifiedColumnsLength" -> userSpecifiedColumnsLength.toString)) } def tableIsNotViewError(name: TableIdentifier): Throwable = { - new AnalysisException(s"$name is not a view") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1278", + messageParameters = Map("name" -> name.toString)) } def viewAlreadyExistsError(name: TableIdentifier): Throwable = { new AnalysisException( - s"View $name already exists. If you want to update the view definition, " + - "please use ALTER VIEW AS or CREATE OR REPLACE VIEW AS") + errorClass = "_LEGACY_ERROR_TEMP_1279", + messageParameters = Map("name" -> name.toString)) } def createPersistedViewFromDatasetAPINotAllowedError(): Throwable = { - new AnalysisException("It is not allowed to create a persisted view from the Dataset API") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1280", + messageParameters = Map.empty) } def recursiveViewDetectedError( viewIdent: TableIdentifier, newPath: Seq[TableIdentifier]): Throwable = { - new AnalysisException(s"Recursive view $viewIdent detected " + - s"(cycle: ${newPath.mkString(" -> ")})") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1281", + messageParameters = Map( + "viewIdent" -> viewIdent.toString, + "newPath" -> newPath.mkString(" -> "))) } def notAllowedToCreatePermanentViewWithoutAssigningAliasForExpressionError( name: TableIdentifier, attrName: String): Throwable = { - new AnalysisException(s"Not allowed to create a permanent view $name without " + - s"explicitly assigning an alias for expression $attrName") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1282", + messageParameters = Map( + "name" -> name.toString, + "attrName" -> attrName)) } def notAllowedToCreatePermanentViewByReferencingTempViewError( name: TableIdentifier, nameParts: String): Throwable = { - new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary view $nameParts. " + - "Please create a temp view instead by CREATE TEMP VIEW") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1283", + messageParameters = Map( + "name" -> name.toString, + "nameParts" -> nameParts)) } def notAllowedToCreatePermanentViewByReferencingTempFuncError( name: TableIdentifier, funcName: String): Throwable = { - new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary function `$funcName`") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1284", + messageParameters = Map( + "name" -> name.toString, + "funcName" -> funcName)) } def queryFromRawFilesIncludeCorruptRecordColumnError(): Throwable = { new AnalysisException( - """ - |Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the - |referenced columns only include the internal corrupt record column - |(named _corrupt_record by default). For example: - |spark.read.schema(schema).csv(file).filter($"_corrupt_record".isNotNull).count() - |and spark.read.schema(schema).csv(file).select("_corrupt_record").show(). - |Instead, you can cache or save the parsed results and then send the same query. - |For example, val df = spark.read.schema(schema).csv(file).cache() and then - |df.filter($"_corrupt_record".isNotNull).count(). - """.stripMargin) + errorClass = "_LEGACY_ERROR_TEMP_1285", + messageParameters = Map.empty) } def userDefinedPartitionNotFoundInJDBCRelationError( columnName: String, schema: String): Throwable = { - new AnalysisException(s"User-defined partition column $columnName not " + - s"found in the JDBC relation: $schema") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1286", + messageParameters = Map( + "columnName" -> columnName, + "schema" -> schema)) } def invalidPartitionColumnTypeError(column: StructField): Throwable = { new AnalysisException( - s""" - |Partition column type should be ${NumericType.simpleString}, - |${DateType.catalogString}, or ${TimestampType.catalogString}, but - |${column.dataType.catalogString} found. - """.stripMargin.replaceAll("\n", " ")) + errorClass = "_LEGACY_ERROR_TEMP_1287", + messageParameters = Map( + "numericType" -> NumericType.simpleString, + "dateType" -> DateType.catalogString, + "timestampType" -> TimestampType.catalogString, + "dataType" -> column.dataType.catalogString)) } def tableOrViewAlreadyExistsError(name: String): Throwable = { new AnalysisException( - s"Table or view '$name' already exists. SaveMode: ErrorIfExists.") + errorClass = "_LEGACY_ERROR_TEMP_1288", + messageParameters = Map("name" -> name)) } def columnNameContainsInvalidCharactersError(name: String): Throwable = { new AnalysisException( - s""" - |Column name "$name" contains invalid character(s). - |Please use alias to rename it. - """.stripMargin.replaceAll("\n", " ")) + errorClass = "_LEGACY_ERROR_TEMP_1289", + messageParameters = Map("name" -> name)) } def textDataSourceWithMultiColumnsError(schema: StructType): Throwable = { new AnalysisException( - s"Text data source supports only a single column, and you have ${schema.size} columns.") + errorClass = "_LEGACY_ERROR_TEMP_1290", + messageParameters = Map("schemaSize" -> schema.size.toString)) } def cannotFindPartitionColumnInPartitionSchemaError( readField: StructField, partitionSchema: StructType): Throwable = { - new AnalysisException(s"Can't find required partition column ${readField.name} " + - s"in partition schema $partitionSchema") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1291", + messageParameters = Map( + "readField" -> readField.name, + "partitionSchema" -> partitionSchema.toString())) } def cannotSpecifyDatabaseForTempViewError(tableIdent: TableIdentifier): Throwable = { new AnalysisException( - s"Temporary view '$tableIdent' should not have specified a database") + errorClass = "_LEGACY_ERROR_TEMP_1292", + messageParameters = Map("tableIdent" -> tableIdent.toString)) } def cannotCreateTempViewUsingHiveDataSourceError(): Throwable = { - new AnalysisException("Hive data source can only be used with tables, " + - "you can't use it with CREATE TEMP VIEW USING") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1293", + messageParameters = Map.empty) } def invalidTimestampProvidedForStrategyError( strategy: String, timeString: String): Throwable = { new AnalysisException( - s"The timestamp provided for the '$strategy' option is invalid. The expected format " + - s"is 'YYYY-MM-DDTHH:mm:ss', but the provided timestamp: $timeString") + errorClass = "_LEGACY_ERROR_TEMP_1294", + messageParameters = Map( + "strategy" -> strategy, + "timeString" -> timeString)) } def hostOptionNotSetError(): Throwable = { - new AnalysisException("Set a host to read from with option(\"host\", ...).") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1295", + messageParameters = Map.empty) } def portOptionNotSetError(): Throwable = { - new AnalysisException("Set a port to read from with option(\"port\", ...).") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1296", + messageParameters = Map.empty) } def invalidIncludeTimestampValueError(): Throwable = { - new AnalysisException("includeTimestamp must be set to either \"true\" or \"false\"") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1297", + messageParameters = Map.empty) } def checkpointLocationNotSpecifiedError(): Throwable = { new AnalysisException( - s""" - |checkpointLocation must be specified either - |through option("checkpointLocation", ...) or - |SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...) - """.stripMargin.replaceAll("\n", " ")) + errorClass = "_LEGACY_ERROR_TEMP_1298", + messageParameters = Map("config" -> SQLConf.CHECKPOINT_LOCATION.key)) } def recoverQueryFromCheckpointUnsupportedError(checkpointPath: Path): Throwable = { - new AnalysisException("This query does not support recovering from checkpoint location. " + - s"Delete $checkpointPath to start over.") + new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_1299", + messageParameters = Map("checkpointPath" -> checkpointPath.toString)) } def cannotFindColumnInRelationOutputError( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index c8d2a002f2a52..2e22e36b926e6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -132,7 +132,7 @@ class EncoderResolutionSuite extends PlanTest { val encoder = ExpressionEncoder[ArrayClass] val attrs = Seq($"arr".array(new StructType().add("c", "int"))) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == - "No such struct field a in c") + "No such struct field a in c.") } test("the real type is not compatible with encoder schema: nested array element type") { @@ -150,8 +150,10 @@ class EncoderResolutionSuite extends PlanTest { withClue("nested array element type is not compatible") { val attrs = Seq($"nestedArr".array(new StructType() .add("arr", ArrayType(new StructType().add("c", "int"))))) - assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == - "No such struct field a in c") + checkError( + exception = intercept[AnalysisException](encoder.resolveAndBind(attrs)), + errorClass = "_LEGACY_ERROR_TEMP_1208", + parameters = Map("fieldName" -> "a", "fields" -> "c")) } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index a39066acbebf7..de87bbb4b9994 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1692,7 +1692,20 @@ select interval (-30) day struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} -- !query @@ -1701,7 +1714,20 @@ select interval (a + 1) day struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} -- !query @@ -1726,7 +1752,20 @@ select interval (-30) days struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} -- !query @@ -1735,7 +1774,20 @@ select interval (a + 1) days struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 72b7601f6f5db..391500dc4e421 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -173,10 +173,13 @@ select 1234567890123456789012345678901234567890 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -decimal can only support precision up to 38 -== SQL == -select 1234567890123456789012345678901234567890 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1229", + "messageParameters" : { + "decimalType" : "decimal", + "precision" : "38" + } +} -- !query @@ -185,10 +188,13 @@ select 1234567890123456789012345678901234567890.0 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -decimal can only support precision up to 38 -== SQL == -select 1234567890123456789012345678901234567890.0 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1229", + "messageParameters" : { + "decimalType" : "decimal", + "precision" : "38" + } +} -- !query @@ -467,7 +473,7 @@ org.apache.spark.sql.catalyst.parser.ParseException { "errorClass" : "_LEGACY_ERROR_TEMP_0061", "messageParameters" : { - "msg" : "decimal can only support precision up to 38" + "msg" : "decimal can only support precision up to 38." }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 9d2ef445e1664..ce07f60a748c7 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -81,7 +81,15 @@ ALTER TABLE test_change CHANGE a TYPE STRING struct<> -- !query output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a' with type 'StringType' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1245", + "messageParameters" : { + "newName" : "a", + "newType" : "StringType", + "originName" : "a", + "originType" : "IntegerType" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out index babce91d99977..ae88227121b66 100644 --- a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out @@ -259,7 +259,15 @@ alter table char_tbl1 change column c type char(6) struct<> -- !query output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'c' with type 'CharType(5)' to 'c' with type 'CharType(6)' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1245", + "messageParameters" : { + "newName" : "c", + "newType" : "CharType(6)", + "originName" : "c", + "originType" : "CharType(5)" + } +} -- !query @@ -575,7 +583,14 @@ alter table char_part partition (v2='ke') rename to partition (v2='nt') struct<> -- !query output org.apache.spark.sql.AnalysisException -Partition spec is invalid. The spec (v2) must match the partition spec (v2, c2) defined in table '`spark_catalog`.`default`.`char_part`' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "v2, c2", + "specKeys" : "v2", + "tableName" : "`spark_catalog`.`default`.`char_part`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out index e6382d7430938..75eb280900679 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out @@ -45,7 +45,13 @@ SELECT * FROM t2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} -- !query @@ -82,7 +88,13 @@ SELECT * FROM t2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} -- !query @@ -136,7 +148,13 @@ SELECT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} -- !query @@ -151,7 +169,13 @@ SELECT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} -- !query @@ -167,7 +191,13 @@ SELECT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} -- !query @@ -181,7 +211,13 @@ WHERE c IN ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} -- !query @@ -210,7 +246,13 @@ SELECT * FROM t struct<> -- !query output org.apache.spark.sql.AnalysisException -Name aBc is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "aBc" + } +} -- !query @@ -223,7 +265,13 @@ SELECT ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Name aBc is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "aBc" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 5bb86a398c82a..5c689cacf4d5f 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -362,9 +362,12 @@ DESC t PARTITION (c='Us', d=2) struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException -Partition not found in table 't' database 'default': -c -> Us -d -> 2 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1238", + "messageParameters" : { + "msg" : "Partition not found in table 't' database 'default':\nc -> Us\nd -> 2" + } +} -- !query @@ -373,7 +376,14 @@ DESC t PARTITION (c='Us') struct<> -- !query output org.apache.spark.sql.AnalysisException -Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`spark_catalog`.`default`.`t`' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "c, d", + "specKeys" : "c", + "tableName" : "`spark_catalog`.`default`.`t`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out index f2eee23a52cdd..6ed91289f9c14 100644 --- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -146,7 +146,20 @@ select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: random_not_exist_func. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.random_not_exist_func.; line 1 pos 29 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.random_not_exist_func", + "rawName" : "random_not_exist_func" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 53, + "fragment" : "random_not_exist_func(1)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 9b91f9a5b8ca0..dc2fd3a06199d 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1573,7 +1573,20 @@ select interval (-30) day struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} -- !query @@ -1582,7 +1595,20 @@ select interval (a + 1) day struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} -- !query @@ -1607,7 +1633,20 @@ select interval (-30) days struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} -- !query @@ -1616,7 +1655,20 @@ select interval (a + 1) days struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.interval", + "rawName" : "interval" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 72b7601f6f5db..391500dc4e421 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -173,10 +173,13 @@ select 1234567890123456789012345678901234567890 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -decimal can only support precision up to 38 -== SQL == -select 1234567890123456789012345678901234567890 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1229", + "messageParameters" : { + "decimalType" : "decimal", + "precision" : "38" + } +} -- !query @@ -185,10 +188,13 @@ select 1234567890123456789012345678901234567890.0 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -decimal can only support precision up to 38 -== SQL == -select 1234567890123456789012345678901234567890.0 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1229", + "messageParameters" : { + "decimalType" : "decimal", + "precision" : "38" + } +} -- !query @@ -467,7 +473,7 @@ org.apache.spark.sql.catalyst.parser.ParseException { "errorClass" : "_LEGACY_ERROR_TEMP_0061", "messageParameters" : { - "msg" : "decimal can only support precision up to 38" + "msg" : "decimal can only support precision up to 38." }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index fcdd42551d1d3..1889f78574a71 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -277,7 +277,13 @@ CREATE VIEW v1_temp AS SELECT * FROM temp_table struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v1_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v1_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -333,7 +339,13 @@ CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v3_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v3_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -382,7 +394,13 @@ CREATE VIEW v4_temp AS struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v4_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v4_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -394,7 +412,13 @@ CREATE VIEW v5_temp AS struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v5_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v5_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -553,7 +577,13 @@ CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v6_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v6_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -562,7 +592,13 @@ CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM tem struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v7_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v7_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -571,7 +607,13 @@ CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v8_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v8_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -580,7 +622,13 @@ CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`temp_view_test`.`v9_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`temp_view_test`.`v9_temp`", + "nameParts" : "temp_table" + } +} -- !query @@ -689,7 +737,13 @@ CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`testviewschm2`.`temporal1` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`testviewschm2`.`temporal1`", + "nameParts" : "tt" + } +} -- !query @@ -730,7 +784,13 @@ CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`testviewschm2`.`temporal2` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`testviewschm2`.`temporal2`", + "nameParts" : "tt" + } +} -- !query @@ -771,7 +831,13 @@ CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`testviewschm2`.`temporal3` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`testviewschm2`.`temporal3`", + "nameParts" : "tt" + } +} -- !query @@ -812,7 +878,13 @@ CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND t struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`testviewschm2`.`temporal4` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`testviewschm2`.`temporal4`", + "nameParts" : "tt" + } +} -- !query @@ -821,7 +893,13 @@ CREATE VIEW temporal5 AS SELECT * FROM t1 WHERE num IN (SELECT num FROM t1 WHERE struct<> -- !query output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `spark_catalog`.`testviewschm2`.`temporal5` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1283", + "messageParameters" : { + "name" : "`spark_catalog`.`testviewschm2`.`temporal5`", + "nameParts" : "tt" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 18226b0fd0397..53a57ee270ba3 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -3581,10 +3581,13 @@ INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.423308199106402476 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -decimal can only support precision up to 38 -== SQL == -INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1229", + "messageParameters" : { + "decimalType" : "decimal", + "precision" : "38" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out index 81f964b4bea07..b3c1e94314d3f 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out @@ -443,7 +443,13 @@ SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS `true` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'm%aca' is invalid, the escape character is not allowed to precede 'a' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'a'", + "pattern" : "m%aca" + } +} -- !query @@ -452,7 +458,13 @@ SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS `false` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'm%aca' is invalid, the escape character is not allowed to precede 'a' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'a'", + "pattern" : "m%aca" + } +} -- !query @@ -461,7 +473,13 @@ SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS `true` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'm%a%%a' is invalid, the escape character is not allowed to precede 'a' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'a'", + "pattern" : "m%a%%a" + } +} -- !query @@ -470,7 +488,13 @@ SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS `false` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'm%a%%a' is invalid, the escape character is not allowed to precede 'a' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'a'", + "pattern" : "m%a%%a" + } +} -- !query @@ -479,7 +503,13 @@ SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS `true` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'b_ear' is invalid, the escape character is not allowed to precede 'e' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'e'", + "pattern" : "b_ear" + } +} -- !query @@ -488,7 +518,13 @@ SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS `false` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'b_ear' is invalid, the escape character is not allowed to precede 'e' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'e'", + "pattern" : "b_ear" + } +} -- !query @@ -497,7 +533,13 @@ SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS `true` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'b_e__r' is invalid, the escape character is not allowed to precede 'e' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'e'", + "pattern" : "b_e__r" + } +} -- !query @@ -506,7 +548,13 @@ SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS `false` struct<> -- !query output org.apache.spark.sql.AnalysisException -the pattern 'b_e__r' is invalid, the escape character is not allowed to precede 'e' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1216", + "messageParameters" : { + "message" : "the escape character is not allowed to precede 'e'", + "pattern" : "b_e__r" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out index d4d8165ddeabe..d9deb9aa31d19 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -407,7 +407,20 @@ SELECT range(1, 100) OVER () FROM empsalary struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: range. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.range.; line 1 pos 7 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.range", + "rawName" : "range" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "range(1, 100) OVER ()" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index f095f98fd8496..7170a8d608150 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -223,7 +223,14 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us') struct<> -- !query output org.apache.spark.sql.AnalysisException -Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`spark_catalog`.`showdb`.`show_t1`' +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "c, d", + "specKeys" : "c", + "tableName" : "`spark_catalog`.`showdb`.`show_t1`" + } +} -- !query @@ -232,7 +239,13 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1) struct<> -- !query output org.apache.spark.sql.AnalysisException -a is not a valid partition column in table `spark_catalog`.`showdb`.`show_t1`. +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1231", + "messageParameters" : { + "key" : "a", + "tblName" : "`spark_catalog`.`showdb`.`show_t1`" + } +} -- !query @@ -241,9 +254,12 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1) struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException -Partition not found in table 'show_t1' database 'showdb': -c -> Ch -d -> 1 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1238", + "messageParameters" : { + "msg" : "Partition not found in table 'show_t1' database 'showdb':\nc -> Ch\nd -> 1" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out index b6c017118a695..688f6103ff379 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out @@ -130,7 +130,20 @@ select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) struct<> -- !query output org.apache.spark.sql.AnalysisException -Undefined function: random_not_exist_func. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.random_not_exist_func.; line 1 pos 42 +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1242", + "messageParameters" : { + "fullName" : "spark_catalog.default.random_not_exist_func", + "rawName" : "random_not_exist_func" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 66, + "fragment" : "random_not_exist_func(1)" + } ] +} -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 9fa9cd16371b2..b13548721b940 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3701,11 +3701,14 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark withTempView("df") { Seq("m@ca").toDF("s").createOrReplaceTempView("df") - val e = intercept[AnalysisException] { - sql("SELECT s LIKE 'm%@ca' ESCAPE '%' FROM df").collect() - } - assert(e.message.contains("the pattern 'm%@ca' is invalid, " + - "the escape character is not allowed to precede '@'")) + checkError( + exception = intercept[AnalysisException] { + sql("SELECT s LIKE 'm%@ca' ESCAPE '%' FROM df").collect() + }, + errorClass = "_LEGACY_ERROR_TEMP_1216", + parameters = Map( + "pattern" -> "m%@ca", + "message" -> "the escape character is not allowed to precede '@'")) checkAnswer(sql("SELECT s LIKE 'm@@ca' ESCAPE '@' FROM df"), Row(true)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 14af2b8241125..91de56334e8fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -525,19 +525,29 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { } test("create table - partition column names not in table definition") { - val e = intercept[AnalysisException] { - sql("CREATE TABLE tbl(a int, b string) USING json PARTITIONED BY (c)") - } - assert(e.message == "partition column c is not defined in table " + - s"$SESSION_CATALOG_NAME.default.tbl, defined table columns are: a, b") + checkError( + exception = intercept[AnalysisException] { + sql("CREATE TABLE tbl(a int, b string) USING json PARTITIONED BY (c)") + }, + errorClass = "_LEGACY_ERROR_TEMP_1206", + parameters = Map( + "colType" -> "partition", + "colName" -> "c", + "tableName" -> s"$SESSION_CATALOG_NAME.default.tbl", + "tableCols" -> "a, b")) } test("create table - bucket column names not in table definition") { - val e = intercept[AnalysisException] { - sql("CREATE TABLE tbl(a int, b string) USING json CLUSTERED BY (c) INTO 4 BUCKETS") - } - assert(e.message == "bucket column c is not defined in table " + - s"$SESSION_CATALOG_NAME.default.tbl, defined table columns are: a, b") + checkError( + exception = intercept[AnalysisException] { + sql("CREATE TABLE tbl(a int, b string) USING json CLUSTERED BY (c) INTO 4 BUCKETS") + }, + errorClass = "_LEGACY_ERROR_TEMP_1206", + parameters = Map( + "colType" -> "bucket", + "colName" -> "c", + "tableName" -> s"$SESSION_CATALOG_NAME.default.tbl", + "tableCols" -> "a, b")) } test("create table - column repeated in partition columns") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala index a8af349a89bd3..d6eea8ae8ed9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala @@ -119,11 +119,15 @@ trait AlterTableSetLocationSuiteBase extends command.AlterTableSetLocationSuiteB sql(buildCreateTableSQL(t)) sql(s"INSERT INTO $t PARTITION (a = '1', b = '2') SELECT 1, 'abc'") - val e = intercept[AnalysisException] { - sql(s"ALTER TABLE $t PARTITION (b='2') SET LOCATION '/mister/spark'") - } - assert(e.getMessage == "Partition spec is invalid. The spec (b) must match the partition " + - "spec (a, b) defined in table '`spark_catalog`.`ns`.`tbl`'") + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t PARTITION (b='2') SET LOCATION '/mister/spark'") + }, + errorClass = "_LEGACY_ERROR_TEMP_1232", + parameters = Map( + "specKeys" -> "b", + "partitionColumnNames" -> "a, b", + "tableName" -> "`spark_catalog`.`ns`.`tbl`")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala index b34d7b03d4768..708c7f7a61880 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala @@ -82,17 +82,19 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu checkSerdeProps(tableIdent, Map.empty[String, String]) // set table serde and/or properties (should fail on datasource tables) - val e1 = intercept[AnalysisException] { - sql(s"ALTER TABLE $t SET SERDE 'whatever'") - } - assert(e1.getMessage == "Operation not allowed: " + - "ALTER TABLE SET SERDE is not supported for tables created with the datasource API") - val e2 = intercept[AnalysisException] { - sql(s"ALTER TABLE $t SET SERDE 'org.apache.madoop' " + - "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") - } - assert(e2.getMessage == "Operation not allowed: " + - "ALTER TABLE SET SERDE is not supported for tables created with the datasource API") + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t SET SERDE 'whatever'") + }, + errorClass = "_LEGACY_ERROR_TEMP_1248", + parameters = Map.empty) + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t SET SERDE 'org.apache.madoop' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + }, + errorClass = "_LEGACY_ERROR_TEMP_1248", + parameters = Map.empty) // set serde properties only sql(s"ALTER TABLE $t SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')") @@ -126,37 +128,37 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu checkPartitionSerdeProps(tableIdent, spec, Map.empty[String, String]) // set table serde and/or properties (should fail on datasource tables) - val e1 = intercept[AnalysisException] { - sql(s"ALTER TABLE $t PARTITION (a=1, b=2) SET SERDE 'whatever'") - } - assert(e1.getMessage == "Operation not allowed: " + - "ALTER TABLE SET [SERDE | SERDEPROPERTIES] for a specific partition " + - "is not supported for tables created with the datasource API") - val e2 = intercept[AnalysisException] { - sql(s"ALTER TABLE $t PARTITION (a=1, b=2) SET SERDE 'org.apache.madoop' " + - "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") - } - assert(e2.getMessage == "Operation not allowed: " + - "ALTER TABLE SET [SERDE | SERDEPROPERTIES] for a specific partition " + - "is not supported for tables created with the datasource API") + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t PARTITION (a=1, b=2) SET SERDE 'whatever'") + }, + errorClass = "_LEGACY_ERROR_TEMP_1247", + parameters = Map.empty) + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t PARTITION (a=1, b=2) SET SERDE 'org.apache.madoop' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + }, + errorClass = "_LEGACY_ERROR_TEMP_1247", + parameters = Map.empty) // set serde properties only - val e3 = intercept[AnalysisException] { - sql(s"ALTER TABLE $t PARTITION (a=1, b=2) " + - "SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')") - } - assert(e3.getMessage == "Operation not allowed: " + - "ALTER TABLE SET [SERDE | SERDEPROPERTIES] for a specific partition " + - "is not supported for tables created with the datasource API") + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t PARTITION (a=1, b=2) " + + "SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')") + }, + errorClass = "_LEGACY_ERROR_TEMP_1247", + parameters = Map.empty) // set things without explicitly specifying database sessionCatalog.setCurrentDatabase("ns") - val e4 = intercept[AnalysisException] { - sql(s"ALTER TABLE tbl PARTITION (a=1, b=2) SET SERDEPROPERTIES ('kay' = 'veee')") - } - assert(e4.getMessage == "Operation not allowed: " + - "ALTER TABLE SET [SERDE | SERDEPROPERTIES] for a specific partition " + - "is not supported for tables created with the datasource API") + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE tbl PARTITION (a=1, b=2) SET SERDEPROPERTIES ('kay' = 'veee')") + }, + errorClass = "_LEGACY_ERROR_TEMP_1247", + parameters = Map.empty) // table to alter does not exist val e5 = intercept[AnalysisException] { From bcae70ca0ae4938e01b71c5a15d5563799ca33a6 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 30 Sep 2022 16:19:58 +0900 Subject: [PATCH 7/8] [SPARK-40619][SQL][TESTS] Add explicitly gc for `HivePartitionFilteringSuite` ### What changes were proposed in this pull request? Similar as SPARK-39869, this pr add explicitly `System.gc()` before each case for `HivePartitionFilteringSuite` to avoid OutOfMemoryError as far as possible. ### Why are the changes needed? Fix flaky test. ``` 2022-09-29T16:23:50.4263170Z [info] org.apache.spark.sql.hive.client.HivePartitionFilteringSuites *** ABORTED *** (26 minutes, 32 seconds) 2022-09-29T16:23:50.4340944Z [info] java.lang.reflect.InvocationTargetException: 2022-09-29T16:23:50.4341736Z [info] at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) 2022-09-29T16:23:50.4342537Z [info] at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) 2022-09-29T16:23:50.4343543Z [info] at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) 2022-09-29T16:23:50.4344319Z [info] at java.lang.reflect.Constructor.newInstance(Constructor.java:423) 2022-09-29T16:23:50.4345108Z [info] at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:315) 2022-09-29T16:23:50.4346070Z [info] at org.apache.spark.sql.hive.client.HiveClientBuilder$.buildClient(HiveClientBuilder.scala:50) 2022-09-29T16:23:50.4347512Z [info] at org.apache.spark.sql.hive.client.HiveVersionSuite.buildClient(HiveVersionSuite.scala:48) 2022-09-29T16:23:50.4348463Z [info] at org.apache.spark.sql.hive.client.HivePartitionFilteringSuite.init(HivePartitionFilteringSuite.scala:73) 2022-09-29T16:23:50.4349656Z [info] at org.apache.spark.sql.hive.client.HivePartitionFilteringSuite.beforeAll(HivePartitionFilteringSuite.scala:118) 2022-09-29T16:23:50.4350533Z [info] at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212) 2022-09-29T16:23:50.4351500Z [info] at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210) 2022-09-29T16:23:50.4352219Z [info] at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208) 2022-09-29T16:23:50.4353147Z [info] at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:66) 2022-09-29T16:23:50.4353841Z [info] at org.scalatest.Suite.callExecuteOnSuite$1(Suite.scala:1178) 2022-09-29T16:23:50.4354737Z [info] at org.scalatest.Suite.$anonfun$runNestedSuites$1(Suite.scala:1225) 2022-09-29T16:23:50.4355475Z [info] at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) 2022-09-29T16:23:50.4356464Z [info] at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33) 2022-09-29T16:23:50.4357212Z [info] at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198) 2022-09-29T16:23:50.4358108Z [info] at org.scalatest.Suite.runNestedSuites(Suite.scala:1223) 2022-09-29T16:23:50.4358777Z [info] at org.scalatest.Suite.runNestedSuites$(Suite.scala:1156) 2022-09-29T16:23:50.4359870Z [info] at org.apache.spark.sql.hive.client.HivePartitionFilteringSuites.runNestedSuites(HivePartitionFilteringSuites.scala:24) 2022-09-29T16:23:50.4360679Z [info] at org.scalatest.Suite.run(Suite.scala:1111) 2022-09-29T16:23:50.4361498Z [info] at org.scalatest.Suite.run$(Suite.scala:1096) 2022-09-29T16:23:50.4362487Z [info] at org.apache.spark.sql.hive.client.HivePartitionFilteringSuites.run(HivePartitionFilteringSuites.scala:24) 2022-09-29T16:23:50.4363571Z [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:321) 2022-09-29T16:23:50.4364320Z [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:516) 2022-09-29T16:23:50.4365208Z [info] at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:413) 2022-09-29T16:23:50.4365870Z [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) 2022-09-29T16:23:50.4366831Z [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 2022-09-29T16:23:50.4368396Z [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 2022-09-29T16:23:50.4368925Z [info] at java.lang.Thread.run(Thread.java:750) 2022-09-29T16:23:50.4369387Z [info] Cause: java.lang.OutOfMemoryError: Metaspace 2022-09-29T16:23:50.4369882Z [info] at java.lang.ClassLoader.defineClass1(Native Method) 2022-09-29T16:23:50.4370399Z [info] at java.lang.ClassLoader.defineClass(ClassLoader.java:757) 2022-09-29T16:23:50.4370965Z [info] at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) 2022-09-29T16:23:50.4371538Z [info] at java.net.URLClassLoader.defineClass(URLClassLoader.java:473) 2022-09-29T16:23:50.4372072Z [info] at java.net.URLClassLoader.access$100(URLClassLoader.java:74) 2022-09-29T16:23:50.4372586Z [info] at java.net.URLClassLoader$1.run(URLClassLoader.java:369) 2022-09-29T16:23:50.4373083Z [info] at java.net.URLClassLoader$1.run(URLClassLoader.java:363) 2022-09-29T16:23:50.4373604Z [info] at java.security.AccessController.doPrivileged(Native Method) 2022-09-29T16:23:50.4374148Z [info] at java.net.URLClassLoader.findClass(URLClassLoader.java:362) 2022-09-29T16:23:50.4374670Z [info] at java.lang.ClassLoader.loadClass(ClassLoader.java:419) 2022-09-29T16:23:50.4375323Z [info] at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.doLoadClass(IsolatedClientLoader.scala:269) 2022-09-29T16:23:50.4376066Z [info] at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.loadClass(IsolatedClientLoader.scala:258) 2022-09-29T16:23:50.4376672Z [info] at java.lang.ClassLoader.loadClass(ClassLoader.java:352) 2022-09-29T16:23:50.4377204Z [info] at org.apache.hadoop.hive.conf.HiveConf.(HiveConf.java:150) 2022-09-29T16:23:50.4377825Z [info] at org.apache.spark.sql.hive.client.HiveClientImpl$.newHiveConf(HiveClientImpl.scala:1284) 2022-09-29T16:23:50.4378489Z [info] at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:171) 2022-09-29T16:23:50.4379144Z [info] at org.apache.spark.sql.hive.client.HiveClientImpl.(HiveClientImpl.scala:136) 2022-09-29T16:23:50.4379774Z [info] at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) 2022-09-29T16:23:50.4380475Z [info] at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) 2022-09-29T16:23:50.4381298Z [info] at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) 2022-09-29T16:23:50.4381965Z [info] at java.lang.reflect.Constructor.newInstance(Constructor.java:423) 2022-09-29T16:23:50.4382648Z [info] at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:315) 2022-09-29T16:23:50.4383383Z [info] at org.apache.spark.sql.hive.client.HiveClientBuilder$.buildClient(HiveClientBuilder.scala:50) 2022-09-29T16:23:50.4384079Z [info] at org.apache.spark.sql.hive.client.HiveVersionSuite.buildClient(HiveVersionSuite.scala:48) 2022-09-29T16:23:50.4384868Z [info] at org.apache.spark.sql.hive.client.HivePartitionFilteringSuite.init(HivePartitionFilteringSuite.scala:73) 2022-09-29T16:23:50.4385686Z [info] at org.apache.spark.sql.hive.client.HivePartitionFilteringSuite.beforeAll(HivePartitionFilteringSuite.scala:118) 2022-09-29T16:23:50.4386560Z [info] at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212) 2022-09-29T16:23:50.4387139Z [info] at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210) 2022-09-29T16:23:50.4387696Z [info] at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208) 2022-09-29T16:23:50.4388245Z [info] at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:66) 2022-09-29T16:23:50.4388781Z [info] at org.scalatest.Suite.callExecuteOnSuite$1(Suite.scala:1178) 2022-09-29T16:23:50.4389309Z [info] at org.scalatest.Suite.$anonfun$runNestedSuites$1(Suite.scala:1225) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions Closes #38057 from LuciferYang/SPARK-40619. Authored-by: yangjie01 Signed-off-by: Hyukjin Kwon --- .../spark/sql/hive/client/HivePartitionFilteringSuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala index efbf0b0b8becb..b96d28d22cc7f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala @@ -39,6 +39,12 @@ import org.apache.spark.util.Utils class HivePartitionFilteringSuite(version: String) extends HiveVersionSuite(version) with BeforeAndAfterAll with SQLHelper { + override def beforeEach(): Unit = { + super.beforeEach() + // SPARK-40619: explicitly call gc to avoid OutOfMemoryError as far as possible. + System.gc() + } + private val tryDirectSqlKey = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname private val fallbackKey = SQLConf.HIVE_METASTORE_PARTITION_PRUNING_FALLBACK_ON_EXCEPTION.key private val pruningFastFallback = SQLConf.HIVE_METASTORE_PARTITION_PRUNING_FAST_FALLBACK.key From 014165ff6ffad87c2d6e970a14c1a13e15ba178f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 30 Sep 2022 19:40:05 +0900 Subject: [PATCH 8/8] [SPARK-40621][PS] Implement `numeric_only` and `min_count` in `GroupBy.sum` ### What changes were proposed in this pull request? Implement `numeric_only` and `min_count` in `GroupBy.sum` ### Why are the changes needed? for API coverage ### Does this PR introduce _any_ user-facing change? new parameter ``` In [2]: df = ps.DataFrame({"A": [1, 2, 1, 2], "B": [True, False, False, True], "C": [3, 4, 3, 4], "D": ["a", "a", "b", "a"]}) In [3]: df.groupby("A").sum(numeric_only=False).sort_index() /Users/ruifeng.zheng/Dev/spark/python/pyspark/pandas/utils.py:975: PandasAPIOnSparkAdviceWarning: GroupBy.sum() can only support numeric and bool columns even ifnumeric_only=False, skip unsupported columns: ['D'] warnings.warn(message, PandasAPIOnSparkAdviceWarning) B C A 1 1 6 2 1 8 In [4]: df._to_pandas().groupby("A").sum(numeric_only=False).sort_index() Out[4]: B C D A 1 1 6 ab 2 1 8 aa In [5]: df.groupby("D").sum(min_count=3).sort_index() Out[5]: A B C D a 5.0 2.0 11.0 b NaN NaN NaN In [6]: df._to_pandas().groupby("D").sum(min_count=3).sort_index() Out[6]: A B C D a 5.0 2.0 11.0 b NaN NaN NaN ``` ### How was this patch tested? added UT Closes #38060 from zhengruifeng/ps_groupby_sum_numonly_mc. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/pandas/groupby.py | 72 +++++++++++++++++++-- python/pyspark/pandas/tests/test_groupby.py | 19 ++++++ 2 files changed, 87 insertions(+), 4 deletions(-) diff --git a/python/pyspark/pandas/groupby.py b/python/pyspark/pandas/groupby.py index 60a791d3deaee..9f459f094a9b5 100644 --- a/python/pyspark/pandas/groupby.py +++ b/python/pyspark/pandas/groupby.py @@ -806,28 +806,92 @@ def std(col: Column) -> Column: bool_to_numeric=True, ) - def sum(self) -> FrameLike: + def sum(self, numeric_only: Optional[bool] = True, min_count: int = 0) -> FrameLike: """ Compute sum of group values + .. versionadded:: 3.3.0 + + Parameters + ---------- + numeric_only : bool, default False + Include only float, int, boolean columns. If None, will attempt to use + everything, then use only numeric data. + It takes no effect since only numeric columns can be support here. + + .. versionadded:: 3.4.0 + min_count: int, default 0 + The required number of valid values to perform the operation. + If fewer than min_count non-NA values are present the result will be NA. + + .. versionadded:: 3.4.0 + Examples -------- >>> df = ps.DataFrame({"A": [1, 2, 1, 2], "B": [True, False, False, True], - ... "C": [3, 4, 3, 4], "D": ["a", "b", "b", "a"]}) + ... "C": [3, 4, 3, 4], "D": ["a", "a", "b", "a"]}) - >>> df.groupby("A").sum() + >>> df.groupby("A").sum().sort_index() B C A 1 1 6 2 1 8 + >>> df.groupby("D").sum().sort_index() + A B C + D + a 5 2 11 + b 1 0 3 + + >>> df.groupby("D").sum(min_count=3).sort_index() + A B C + D + a 5.0 2.0 11.0 + b NaN NaN NaN + + Notes + ----- + There is a behavior difference between pandas-on-Spark and pandas: + + * when there is a non-numeric aggregation column, it will be ignored + even if `numeric_only` is False. + See Also -------- pyspark.pandas.Series.groupby pyspark.pandas.DataFrame.groupby """ + if numeric_only is not None and not isinstance(numeric_only, bool): + raise TypeError("numeric_only must be None or bool") + if not isinstance(min_count, int): + raise TypeError("min_count must be integer") + + if numeric_only is not None and not numeric_only: + unsupported = [ + col.name + for col in self._agg_columns + if not isinstance(col.spark.data_type, (NumericType, BooleanType)) + ] + if len(unsupported) > 0: + log_advice( + "GroupBy.sum() can only support numeric and bool columns even if" + f"numeric_only=False, skip unsupported columns: {unsupported}" + ) + + if min_count > 0: + + def sum(col: Column) -> Column: + return F.when( + F.count(F.when(~F.isnull(col), F.lit(0))) < min_count, F.lit(None) + ).otherwise(F.sum(col)) + + else: + + def sum(col: Column) -> Column: + return F.sum(col) + return self._reduce_for_stat_function( - F.sum, accepted_spark_types=(NumericType,), bool_to_numeric=True + sum, accepted_spark_types=(NumericType,), bool_to_numeric=True ) # TODO: sync the doc. diff --git a/python/pyspark/pandas/tests/test_groupby.py b/python/pyspark/pandas/tests/test_groupby.py index 14e699793f318..ba283cc9a0564 100644 --- a/python/pyspark/pandas/tests/test_groupby.py +++ b/python/pyspark/pandas/tests/test_groupby.py @@ -1413,6 +1413,25 @@ def test_max(self): self._test_stat_func(lambda groupby_obj: groupby_obj.max(numeric_only=True)) self._test_stat_func(lambda groupby_obj: groupby_obj.max(numeric_only=True, min_count=2)) + def test_sum(self): + pdf = pd.DataFrame( + { + "A": ["a", "a", "b", "a"], + "B": [1, 2, 1, 2], + "C": [-1.5, np.nan, -3.2, 0.1], + } + ) + psdf = ps.from_pandas(pdf) + self.assert_eq(pdf.groupby("A").sum().sort_index(), psdf.groupby("A").sum().sort_index()) + self.assert_eq( + pdf.groupby("A").sum(min_count=2).sort_index(), + psdf.groupby("A").sum(min_count=2).sort_index(), + ) + self.assert_eq( + pdf.groupby("A").sum(min_count=3).sort_index(), + psdf.groupby("A").sum(min_count=3).sort_index(), + ) + def test_mad(self): self._test_stat_func(lambda groupby_obj: groupby_obj.mad())