juliuszsompolski commented on code in PR #41315: URL: https://github.com/apache/spark/pull/41315#discussion_r1205839099
########## connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecutionHolder.scala: ########## @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.service + +import scala.util.control.NonFatal + +import com.google.protobuf.Message +import io.grpc.stub.StreamObserver +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.SparkSQLException +import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.{ExecutePlanRequest, ExecutePlanResponse} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager +import org.apache.spark.sql.connect.common.ProtoUtils +import org.apache.spark.sql.connect.execution.{ExecutePlanResponseObserver, SparkConnectPlanExecution} +import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.util.Utils + +/** + * Object used to hold the Spark Connect execution state, and perform + */ +case class ExecutionHolder(operationId: String, sessionHolder: SessionHolder) extends Logging { + + val jobGroupId = + s"User_${sessionHolder.userId}_Session_${sessionHolder.sessionId}_Request_${operationId}" + + val session = sessionHolder.session + + var executePlanRequest: Option[proto.ExecutePlanRequest] = None + + var executePlanResponseObserver: Option[ExecutePlanResponseObserver] = None Review Comment: I renamed it, and made executePlanRequest / executePlanResponseObserver options here, because I think this ExecutionHolder could be useful for holding other executions than ExecutePlanRequest. E.g. we might want to execute requests from AnalyzePlanRequest using this as well... ########## connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala: ########## @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.execution + +import scala.collection.JavaConverters._ + +import com.google.protobuf.ByteString +import io.grpc.stub.StreamObserver + +import org.apache.spark.SparkEnv +import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.ExecutePlanResponse +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connect.common.DataTypeProtoConverter +import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto +import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE +import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.sql.connect.service.ExecutionHolder +import org.apache.spark.sql.execution.{LocalTableScanExec, SQLExecution} +import org.apache.spark.sql.execution.arrow.ArrowConverters +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ThreadUtils + +/** + * Handle ExecutePlanRequest where the operatoin to handle is Plan execution of type + * proto.Plan.OpTypeCase.ROOT + * @param executionHolder + */ +class SparkConnectPlanExecution(executionHolder: ExecutionHolder) { Review Comment: This class is no-changes verbatim moved out of SparkConnectStreamHandler. ########## connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala: ########## @@ -15,27 +15,32 @@ * limitations under the License. */ -package org.apache.spark.sql.connect.service +package org.apache.spark.sql.connect.execution -import org.apache.spark.connect.proto +import io.grpc.stub.StreamObserver + +import org.apache.spark.connect.proto.ExecutePlanResponse /** - * Object used to hold the Spark Connect execution state. + * Container for responses to Execution. + * TODO: At this moment, it simply forwards response to the underlying GRPC StreamObserver. + * Detaching execution from the RPC handler, this can be used to store the responses, + * and then send them to different GRPC StreamObservers. + * + * @param responseObserver */ -case class ExecutePlanHolder( - operationId: String, - sessionHolder: SessionHolder, - request: proto.ExecutePlanRequest) { +class ExecutePlanResponseObserver(responseObserver: StreamObserver[ExecutePlanResponse]) + extends StreamObserver[ExecutePlanResponse] { Review Comment: This is a bit of a stub for now, but wanted to put it in already, so I don't need to move code around in followups. ########## connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/executionUtils.scala: ########## @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.execution + +import scala.collection.JavaConverters._ + +import org.apache.spark.connect.proto.ExecutePlanResponse +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, QueryStageExec} + +/** + * Helper object for generating responses with metrics from queries. + */ +object MetricGenerator extends AdaptiveSparkPlanHelper { Review Comment: This is practically no-changes moved out of SparkConnectStreamHandler. ########## connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecutionHolder.scala: ########## @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.service + +import scala.util.control.NonFatal + +import com.google.protobuf.Message +import io.grpc.stub.StreamObserver +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.SparkSQLException +import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.{ExecutePlanRequest, ExecutePlanResponse} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager +import org.apache.spark.sql.connect.common.ProtoUtils +import org.apache.spark.sql.connect.execution.{ExecutePlanResponseObserver, SparkConnectPlanExecution} +import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.util.Utils + +/** + * Object used to hold the Spark Connect execution state, and perform + */ +case class ExecutionHolder(operationId: String, sessionHolder: SessionHolder) extends Logging { + + val jobGroupId = + s"User_${sessionHolder.userId}_Session_${sessionHolder.sessionId}_Request_${operationId}" + + val session = sessionHolder.session + + var executePlanRequest: Option[proto.ExecutePlanRequest] = None + + var executePlanResponseObserver: Option[ExecutePlanResponseObserver] = None + + private var executionThread: Thread = null + + private var executionError: Option[Throwable] = None + + private var interrupted: Boolean = false + + def run( + request: proto.ExecutePlanRequest, + responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { + // Set the state of what needs to be run. + this.executePlanRequest = Some(request) + this.executePlanResponseObserver = Some(new ExecutePlanResponseObserver(responseObserver)) + // And start the execution. + startExecute() + } + + protected def startExecute(): Unit = { + // synchronized in case of interrupt while starting. + synchronized { + // The newly created thread will inherit all InheritableThreadLocals used by Spark, + // e.g. SparkContext.localProperties./ If considering implementing a threadpool, + // forwarding of thread locals needs to be taken into account. + this.executionThread = new Thread() { + override def run(): Unit = { + execute() + } + } + } + + try { + // Start execution thread.. + this.executionThread.start() + // ... and wait for execution thread to finish. + // TODO: Detach execution from RPC request. Then this can return early, and results + // are served to the client via additional RPCs from ExecutePlanResponseObserver. + this.executionThread.join() + + executionError.foreach { error => + logDebug(s"executionError: ${error}") + throw error + } + } catch { + case NonFatal(e) => + // In case of exception happening on the handler thread, interrupt the underlying execution. + this.interrupt() + throw e + } + } + + protected def execute() = SparkConnectArtifactManager.withArtifactClassLoader { + try { + // synchronized - check if already got interrupted while starting. + synchronized { + if (interrupted) { + throw new InterruptedException() + } + } + + session.withActive { + val debugString = requestString(executePlanRequest.get) + + session.sparkContext.setJobGroup( + jobGroupId, + s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}", + interruptOnCancel = true) + + // Add debug information to the query execution so that the jobs are traceable. + session.sparkContext.setLocalProperty( + "callSite.short", + s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}") + session.sparkContext.setLocalProperty( + "callSite.long", + StringUtils.abbreviate(debugString, 2048)) + + executePlanRequest.foreach { request => + request.getPlan.getOpTypeCase match { + case proto.Plan.OpTypeCase.COMMAND => handleCommand(request) + case proto.Plan.OpTypeCase.ROOT => handlePlan(request) + case _ => + throw new UnsupportedOperationException( + s"${request.getPlan.getOpTypeCase} not supported.") + } + } + } + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + // scalastyle:off + logDebug(s"Exception in execute: $e") + // Always cancel all remaining execution after error. + sessionHolder.session.sparkContext.cancelJobGroup(jobGroupId) + executionError = if (interrupted) { + // Turn the interrupt into OPERATION_CANCELLED error. + Some(new SparkSQLException("OPERATION_CANCELLED", Map.empty)) + } else { + // Return the originally thrown error. + Some(e) + } + } finally { + session.sparkContext.clearJobGroup() + } + } + + def interrupt(): Unit = { + synchronized { + interrupted = true + if (executionThread != null) { + executionThread.interrupt() + } + } + } + + private def handlePlan(request: ExecutePlanRequest): Unit = { + val request = executePlanRequest.get + val responseObserver = executePlanResponseObserver.get + + val execution = new SparkConnectPlanExecution(this) + execution.handlePlan(responseObserver) + } + + private def handleCommand(request: ExecutePlanRequest): Unit = { + val request = executePlanRequest.get + val responseObserver = executePlanResponseObserver.get + + val command = request.getPlan.getCommand + val planner = new SparkConnectPlanner(sessionHolder.session) Review Comment: I think command execution should be refactored into org.apache.spark.sql.connect.execution.SparkConnectCommandExecution in a followup. It doesn't belong in SparkConnectPlanner. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
