Skip to content

[WIP][SPARK-53292] Make CreateResourceProfileCommand in SparkConnectPlanner side effect free #52064

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import org.apache.spark.sql.classic.{DataFrame, Dataset}
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.execution.command.ConnectLeafRunnableCommand
import org.apache.spark.sql.connect.planner.{InvalidInputErrors, SparkConnectPlanner}
import org.apache.spark.sql.connect.service.ExecuteHolder
import org.apache.spark.sql.connect.utils.MetricGenerator
Expand Down Expand Up @@ -90,6 +91,15 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder)
shuffleCleanupMode = shuffleCleanupMode)
qe.assertCommandExecuted()
executeHolder.eventsManager.postFinished()
qe.logical match {
case connectCommand: ConnectLeafRunnableCommand =>
connectCommand.handleConnectResponse(
responseObserver,
sessionHolder.sessionId,
sessionHolder.serverSessionId)
case _ =>
// Do nothing
}
case None =>
planner.process(command, responseObserver)
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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.command

import io.grpc.stub.StreamObserver

import org.apache.spark.connect.proto
import org.apache.spark.sql.{classic, Row, SparkSession}
import org.apache.spark.sql.execution.command.LeafRunnableCommand

trait ConnectLeafRunnableCommand extends LeafRunnableCommand {

final override def run(sparkSession: SparkSession): Seq[Row] = {
run(sparkSession.asInstanceOf[classic.SparkSession])
}

def run(sparkSession: classic.SparkSession): Seq[Row]

def handleConnectResponse(
responseObserver: StreamObserver[proto.ExecutePlanResponse],
sessionId: String,
serverSessionId: String): Unit = {
// Default implementation does nothing.
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.command

import io.grpc.stub.StreamObserver

import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.ExecutePlanResponse
import org.apache.spark.resource.ResourceProfile
import org.apache.spark.sql.Row
import org.apache.spark.sql.classic.SparkSession

case class CreateResourceProfileCommand(rp: ResourceProfile) extends ConnectLeafRunnableCommand {

override def run(sparkSession: SparkSession): Seq[Row] = {
sparkSession.sparkContext.resourceProfileManager.addResourceProfile(rp)
Seq.empty
}

override def handleConnectResponse(
responseObserver: StreamObserver[ExecutePlanResponse],
sessionId: String,
serverSessionId: String): Unit = {
responseObserver.onNext(
proto.ExecutePlanResponse
.newBuilder()
.setSessionId(sessionId)
.setServerSideSessionId(serverSessionId)
.setCreateResourceProfileCommandResult(
proto.CreateResourceProfileCommandResult
.newBuilder()
.setProfileId(rp.id)
.build())
.build())
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.apache.spark.{SparkClassNotFoundException, SparkEnv, SparkException,
import org.apache.spark.annotation.{DeveloperApi, Since}
import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{CheckpointCommand, CreateResourceProfileCommand, ExecutePlanResponse, SqlCommand, StreamingForeachFunction, StreamingQueryCommand, StreamingQueryCommandResult, StreamingQueryInstanceId, StreamingQueryManagerCommand, StreamingQueryManagerCommandResult, WriteStreamOperationStart, WriteStreamOperationStartResult}
import org.apache.spark.connect.proto.{CheckpointCommand, ExecutePlanResponse, SqlCommand, StreamingForeachFunction, StreamingQueryCommand, StreamingQueryCommandResult, StreamingQueryInstanceId, StreamingQueryManagerCommand, StreamingQueryManagerCommandResult, WriteStreamOperationStart, WriteStreamOperationStartResult}
import org.apache.spark.connect.proto.ExecutePlanResponse.SqlCommandResult
import org.apache.spark.connect.proto.Parse.ParseFormat
import org.apache.spark.connect.proto.StreamingQueryManagerCommandResult.StreamingQueryInstance
Expand Down Expand Up @@ -62,6 +62,7 @@ import org.apache.spark.sql.classic.ClassicConversions._
import org.apache.spark.sql.connect.client.arrow.ArrowSerializer
import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket}
import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE
import org.apache.spark.sql.connect.execution.command.{ConnectLeafRunnableCommand, CreateResourceProfileCommand}
import org.apache.spark.sql.connect.ml.MLHandler
import org.apache.spark.sql.connect.pipelines.PipelinesHandler
import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry
Expand Down Expand Up @@ -2654,6 +2655,8 @@ class SparkConnectPlanner(
Some(transformMergeIntoTableCommand(command.getMergeIntoTableCommand))
case proto.Command.CommandTypeCase.CREATE_DATAFRAME_VIEW =>
Some(_ => transformCreateViewCommand(command.getCreateDataframeView))
case proto.Command.CommandTypeCase.CREATE_RESOURCE_PROFILE_COMMAND =>
Some(_ => transformCreateResourceProfileCommand(command.getCreateResourceProfileCommand))
case _ =>
None
}
Expand All @@ -2664,7 +2667,7 @@ class SparkConnectPlanner(
responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
val transformerOpt = transformCommand(command)
if (transformerOpt.isDefined) {
transformAndRunCommand(transformerOpt.get)
transformAndRunCommand(transformerOpt.get, responseObserver)
return
}
command.getCommandTypeCase match {
Expand Down Expand Up @@ -2693,10 +2696,6 @@ class SparkConnectPlanner(
responseObserver)
case proto.Command.CommandTypeCase.GET_RESOURCES_COMMAND =>
handleGetResourcesCommand(responseObserver)
case proto.Command.CommandTypeCase.CREATE_RESOURCE_PROFILE_COMMAND =>
handleCreateResourceProfileCommand(
command.getCreateResourceProfileCommand,
responseObserver)
case proto.Command.CommandTypeCase.CHECKPOINT_COMMAND =>
handleCheckpointCommand(command.getCheckpointCommand, responseObserver)
case proto.Command.CommandTypeCase.REMOVE_CACHED_REMOTE_RELATION_COMMAND =>
Expand Down Expand Up @@ -3153,11 +3152,22 @@ class SparkConnectPlanner(
}
}

private def transformAndRunCommand(transformer: QueryPlanningTracker => LogicalPlan): Unit = {
private def transformAndRunCommand(
transformer: QueryPlanningTracker => LogicalPlan,
responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val qe = new QueryExecution(session, transformer(tracker), tracker)
qe.assertCommandExecuted()
executeHolder.eventsManager.postFinished()
qe.logical match {
case connectCommand: ConnectLeafRunnableCommand =>
connectCommand.handleConnectResponse(
responseObserver,
sessionId,
sessionHolder.serverSessionId)
case _ =>
// Do nothing
}
}

/**
Expand Down Expand Up @@ -3674,9 +3684,8 @@ class SparkConnectPlanner(
.build())
}

private def handleCreateResourceProfileCommand(
createResourceProfileCommand: CreateResourceProfileCommand,
responseObserver: StreamObserver[proto.ExecutePlanResponse]): Unit = {
private def transformCreateResourceProfileCommand(
createResourceProfileCommand: proto.CreateResourceProfileCommand): LogicalPlan = {
val rp = createResourceProfileCommand.getProfile
val ereqs = rp.getExecutorResourcesMap.asScala.map { case (name, res) =>
name -> new ExecutorResourceRequest(
Expand All @@ -3695,20 +3704,7 @@ class SparkConnectPlanner(
} else {
new ResourceProfile(ereqs, treqs)
}
session.sparkContext.resourceProfileManager.addResourceProfile(profile)

executeHolder.eventsManager.postFinished()
responseObserver.onNext(
proto.ExecutePlanResponse
.newBuilder()
.setSessionId(sessionId)
.setServerSideSessionId(sessionHolder.serverSessionId)
.setCreateResourceProfileCommandResult(
proto.CreateResourceProfileCommandResult
.newBuilder()
.setProfileId(profile.id)
.build())
.build())
CreateResourceProfileCommand(profile)
}

private def handleCheckpointCommand(
Expand Down