[ https://issues.apache.org/jira/browse/SPARK-51451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Bobby Wang updated SPARK-51451: ------------------------------- Affects Version/s: 4.0.0 > [UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS] The generator is not supported: > nested in expressions "unresolvedstarwithcolumns(explode(array(0, 1, 2, 3, 4, > 5, 6, 7, 8, 9, 10, 11)))". SQLSTATE: 42K0E > ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ > > Key: SPARK-51451 > URL: https://issues.apache.org/jira/browse/SPARK-51451 > Project: Spark > Issue Type: Bug > Components: Connect, SQL > Affects Versions: 4.0.0, 4.1.0 > Reporter: Bobby Wang > Priority: Major > > The below code could run on classic spark. but failed to run on connect spark. > > {code:java} > from pyspark.sql import SparkSession > from pyspark.sql.functions import * > spark = (SparkSession > .builder > .remote("sc://localhost") > .getOrCreate()) > df = spark.createDataFrame([("082017",)], ['dt']) > df_dt = df.select(date_format(to_date(col("dt"), "MMyyyy"), > "MM/dd/yyyy").alias("dt")) > monthArray = [lit(x) for x in range(0, 12)] > df_month_y = df_dt.withColumn("month_y", explode(array(monthArray))) > df_month_y.show(){code} > > Spark connect throws the below exception > > pyspark.errors.exceptions.connect.AnalysisException: > [UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS] The generator is not supported: > nested in expressions "unresolvedstarwithcolumns(explode(array(0, 1, 2, 3, 4, > 5, 6, 7, 8, 9, 10, 11)))". SQLSTATE: 42K0E > > JVM stacktrace: > org.apache.spark.sql.AnalysisException > at > org.apache.spark.sql.errors.QueryCompilationErrors$.nestedGeneratorError(QueryCompilationErrors.scala:315) > at > org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator$$anonfun$apply$35.applyOrElse(Analyzer.scala:2914) > at > org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator$$anonfun$apply$35.applyOrElse(Analyzer.scala:2911) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:138) > at > org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:138) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:386) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:134) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:130) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:37) > at > org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator$.apply(Analyzer.scala:2911) > at > org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator$.apply(Analyzer.scala:2839) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) > at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) > at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) > at scala.collection.immutable.List.foldLeft(List.scala:79) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) > at scala.collection.immutable.List.foreach(List.scala:334) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) > at > org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:288) > at > org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:284) > at > org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:232) > at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:284) > at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:247) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) > at > org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) > at > org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.resolveInFixedPoint(HybridAnalyzer.scala:190) > at > org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.$anonfun$apply$1(HybridAnalyzer.scala:76) > at > org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.withTrackedAnalyzerBridgeState(HybridAnalyzer.scala:111) > at > org.apache.spark.sql.catalyst.analysis.resolver.HybridAnalyzer.apply(HybridAnalyzer.scala:71) > at > org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:278) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:393) > at > org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:278) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$2(QueryExecution.scala:99) > at > org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:148) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:267) > at > org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:643) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:267) > at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804) > at > org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:266) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyAnalyzed$1(QueryExecution.scala:99) > at scala.util.Try$.apply(Try.scala:217) > at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1378) > at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1439) > at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) > at > org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:110) > at > org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:79) > at org.apache.spark.sql.classic.Dataset$.$anonfun$ofRows$1(Dataset.scala:115) > at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804) > at org.apache.spark.sql.classic.Dataset$.ofRows(Dataset.scala:113) > at > org.apache.spark.sql.connect.planner.SparkConnectPlanner.transformShowString(SparkConnectPlanner.scala:306) > at > org.apache.spark.sql.connect.planner.SparkConnectPlanner.$anonfun$transformRelation$1(SparkConnectPlanner.scala:150) > at > org.apache.spark.sql.connect.service.SessionHolder.$anonfun$usePlanCache$3(SessionHolder.scala:477) > at scala.Option.getOrElse(Option.scala:201) > at > org.apache.spark.sql.connect.service.SessionHolder.usePlanCache(SessionHolder.scala:476) > at > org.apache.spark.sql.connect.planner.SparkConnectPlanner.transformRelation(SparkConnectPlanner.scala:147) > at > org.apache.spark.sql.connect.execution.SparkConnectPlanExecution.handlePlan(SparkConnectPlanExecution.scala:74) > at > org.apache.spark.sql.connect.execution.ExecuteThreadRunner.handlePlan(ExecuteThreadRunner.scala:314) > at > org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1(ExecuteThreadRunner.scala:225) > at > org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1$adapted(ExecuteThreadRunner.scala:196) > at > org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$2(SessionHolder.scala:341) > at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804) > at > org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$1(SessionHolder.scala:341) > at > org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94) > at > org.apache.spark.sql.artifact.ArtifactManager.$anonfun$withResources$1(ArtifactManager.scala:112) > at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:186) > at > org.apache.spark.sql.artifact.ArtifactManager.withClassLoaderIfNeeded(ArtifactManager.scala:102) > at > org.apache.spark.sql.artifact.ArtifactManager.withResources(ArtifactManager.scala:111) > at > org.apache.spark.sql.connect.service.SessionHolder.withSession(SessionHolder.scala:340) > at > org.apache.spark.sql.connect.execution.ExecuteThreadRunner.executeInternal(ExecuteThreadRunner.scala:196) > at > org.apache.spark.sql.connect.execution.ExecuteThreadRunner.org$apache$spark$sql$connect$execution$ExecuteThreadRunner$$execute(ExecuteThreadRunner.scala:125) > at > org.apache.spark.sql.connect.execution.ExecuteThreadRunner$ExecutionThread.run(ExecuteThreadRunner.scala:347) -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org