jonmio commented on code in PR #51057: URL: https://github.com/apache/spark/pull/51057#discussion_r2136627232
########## sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala: ########## @@ -0,0 +1,288 @@ +/* + * 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.pipelines + +import scala.jdk.CollectionConverters._ + +import com.google.protobuf.{Timestamp => ProtoTimestamp} +import io.grpc.stub.StreamObserver + +import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.{ExecutePlanResponse, PipelineCommandResult, Relation} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.classic.SparkSession +import org.apache.spark.sql.connect.common.DataTypeProtoConverter +import org.apache.spark.sql.connect.service.SessionHolder +import org.apache.spark.sql.pipelines.Language.Python +import org.apache.spark.sql.pipelines.QueryOriginType +import org.apache.spark.sql.pipelines.common.RunState.{CANCELED, FAILED} +import org.apache.spark.sql.pipelines.graph.{FlowAnalysis, GraphIdentifierManager, IdentifierHelper, PipelineUpdateContextImpl, QueryContext, QueryOrigin, SqlGraphRegistrationContext, Table, TemporaryView, UnresolvedFlow} +import org.apache.spark.sql.pipelines.logging.{PipelineEvent, RunProgress} +import org.apache.spark.sql.types.StructType + +/** Handler for SparkConnect PipelineCommands */ +private[connect] object PipelinesHandler extends Logging { + + /** + * Handles the pipeline command + * @param sessionHolder + * Context about the session state + * @param cmd + * Command to be handled + * @param responseObserver + * The response observer where the response will be sent + * @param sparkSession + * The spark session + * @param transformRelationFunc + * Function used to convert a relation to a LogicalPlan. This is used when determining the + * LogicalPlan that a flow returns. + * @return + * The response after handling the command + */ + def handlePipelinesCommand( + sessionHolder: SessionHolder, + cmd: proto.PipelineCommand, + responseObserver: StreamObserver[ExecutePlanResponse], + sparkSession: SparkSession, + transformRelationFunc: Relation => LogicalPlan): PipelineCommandResult = { + // Currently most commands do not include any information in the response. We just send back + // an empty response to the client to indicate that the command was handled successfully + val defaultResponse = PipelineCommandResult.getDefaultInstance + cmd.getCommandTypeCase match { + case proto.PipelineCommand.CommandTypeCase.CREATE_DATAFLOW_GRAPH => + val createdGraphId = createDataflowGraph(cmd.getCreateDataflowGraph) + PipelineCommandResult + .newBuilder() + .setCreateDataflowGraphResult( + PipelineCommandResult.CreateDataflowGraphResult.newBuilder + .setDataflowGraphId(createdGraphId) + .build()) + .build() + case proto.PipelineCommand.CommandTypeCase.DROP_DATAFLOW_GRAPH => + logInfo(s"Drop pipeline cmd received: $cmd") + DataflowGraphRegistry.dropDataflowGraph(cmd.getDropDataflowGraph.getDataflowGraphId) + defaultResponse + case proto.PipelineCommand.CommandTypeCase.DEFINE_DATASET => + logInfo(s"Define pipelines dataset cmd received: $cmd") + defineDataset(cmd.getDefineDataset, sparkSession) + defaultResponse + case proto.PipelineCommand.CommandTypeCase.DEFINE_FLOW => + logInfo(s"Define pipelines flow cmd received: $cmd") + defineFlow(cmd.getDefineFlow, transformRelationFunc, sparkSession) + defaultResponse + case proto.PipelineCommand.CommandTypeCase.START_RUN => + logInfo(s"Start pipeline cmd received: $cmd") + startRun(cmd.getStartRun, responseObserver, sessionHolder) + defaultResponse + case proto.PipelineCommand.CommandTypeCase.DEFINE_SQL_GRAPH_ELEMENTS => + logInfo(s"Register sql datasets cmd received: $cmd") + defineSqlGraphElements(cmd.getDefineSqlGraphElements, sparkSession) + defaultResponse + case other => throw new UnsupportedOperationException(s"$other not supported") + } + } + + private def createDataflowGraph(cmd: proto.PipelineCommand.CreateDataflowGraph): String = { + val defaultCatalog = Option + .when(cmd.hasDefaultCatalog)(cmd.getDefaultCatalog) + .getOrElse { + logInfo( + s"No default catalog was supplied. Falling back to the session catalog `spark_catalog`).") + "spark_catalog" + } + + val defaultDatabase = Option + .when(cmd.hasDefaultDatabase)(cmd.getDefaultDatabase) + .getOrElse { + logInfo(s"No default catalog was supplied. Falling back to `default`).") + "default" + } + + val defaultSqlConf = cmd.getSqlConfMap.asScala.toMap + + DataflowGraphRegistry.createDataflowGraph( + defaultCatalog = defaultCatalog, + defaultDatabase = defaultDatabase, + defaultSqlConf = defaultSqlConf) + } + + private def defineSqlGraphElements( + cmd: proto.PipelineCommand.DefineSqlGraphElements, + session: SparkSession): Unit = { + val dataflowGraphId = cmd.getDataflowGraphId + + val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId) + val sqlGraphElementRegistrationContext = new SqlGraphRegistrationContext(graphElementRegistry) + sqlGraphElementRegistrationContext.processSqlFile(cmd.getSqlText, cmd.getSqlFilePath, session) + } + + private def defineDataset( + dataset: proto.PipelineCommand.DefineDataset, + sparkSession: SparkSession): Unit = { + val dataflowGraphId = dataset.getDataflowGraphId + val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId) + + dataset.getDatasetType match { + case proto.DatasetType.MATERIALIZED_VIEW | proto.DatasetType.TABLE => + val tableIdentifier = + GraphIdentifierManager.parseTableIdentifier(dataset.getDatasetName, sparkSession) + graphElementRegistry.registerTable( + Table( + identifier = tableIdentifier, + comment = Option(dataset.getComment), + specifiedSchema = Option.when(dataset.hasSchema)( + DataTypeProtoConverter + .toCatalystType(dataset.getSchema) + .asInstanceOf[StructType]), + partitionCols = Option(dataset.getPartitionColsList.asScala.toSeq) + .filter(_.nonEmpty), + properties = dataset.getTablePropertiesMap.asScala.toMap, + baseOrigin = QueryOrigin( + objectType = Option(QueryOriginType.Table.toString), + objectName = Option(tableIdentifier.unquotedString), + language = Option(Python())), + format = Option.when(dataset.hasFormat)(dataset.getFormat), + normalizedPath = None, + isStreamingTableOpt = None)) + case proto.DatasetType.TEMPORARY_VIEW => + val viewIdentifier = + GraphIdentifierManager.parseTableIdentifier(dataset.getDatasetName, sparkSession) + + graphElementRegistry.registerView( + TemporaryView( + identifier = viewIdentifier, + comment = Option(dataset.getComment), + origin = QueryOrigin( + objectType = Option(QueryOriginType.View.toString), + objectName = Option(viewIdentifier.unquotedString), + language = Option(Python())), + properties = Map.empty)) + case _ => + throw new IllegalArgumentException(s"Unknown dataset type: ${dataset.getDatasetType}") + } + } + + private def defineFlow( + flow: proto.PipelineCommand.DefineFlow, + transformRelationFunc: Relation => LogicalPlan, + sparkSession: SparkSession): Unit = { + val dataflowGraphId = flow.getDataflowGraphId + val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId) + + val isImplicitFlow = flow.getFlowName == flow.getTargetDatasetName + + val flowIdentifier = GraphIdentifierManager + .parseTableIdentifier(name = flow.getFlowName, spark = sparkSession) + + // If the flow is not an implicit flow (i.e. one defined as part of dataset creation), then + // it must be a single-part identifier. + if (!isImplicitFlow && !IdentifierHelper.isSinglePartIdentifier(flowIdentifier)) { + throw new AnalysisException( + "MULTIPART_FLOW_NAME_NOT_SUPPORTED", + Map("flowName" -> flow.getFlowName)) + } + + graphElementRegistry.registerFlow( + new UnresolvedFlow( + identifier = flowIdentifier, + destinationIdentifier = GraphIdentifierManager + .parseTableIdentifier(name = flow.getTargetDatasetName, spark = sparkSession), + func = + FlowAnalysis.createFlowFunctionFromLogicalPlan(transformRelationFunc(flow.getPlan)), + sqlConf = flow.getSqlConfMap.asScala.toMap, + once = flow.getOnce, + queryContext = QueryContext( + Option(graphElementRegistry.defaultCatalog), + Option(graphElementRegistry.defaultDatabase)), + comment = None, + origin = QueryOrigin( + objectType = Option(QueryOriginType.Flow.toString), + objectName = Option(flowIdentifier.unquotedString), + language = Option(Python())))) + } + + private def startRun( + cmd: proto.PipelineCommand.StartRun, + responseObserver: StreamObserver[ExecutePlanResponse], + sessionHolder: SessionHolder): Unit = { + val dataflowGraphId = cmd.getDataflowGraphId + val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId) + // We will use this variable to store the run failure event if it occurs. This will be set + // by the event callback. + var runFailureEvent = Option.empty[PipelineEvent] Review Comment: Yup marked as volatile since this can be called by other threads that add events to the buffer causing the callback to be invoked -- 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: reviews-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org