-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-45204][CONNECT] Add optional ExecuteHolder to SparkConnectPlanner #43311
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -86,14 +86,29 @@ final case class InvalidCommandInput( | |
| private val cause: Throwable = null) | ||
| extends Exception(message, cause) | ||
|
|
||
| class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | ||
| class SparkConnectPlanner( | ||
| val sessionHolder: SessionHolder, | ||
| val executeHolderOpt: Option[ExecuteHolder] = None) | ||
| extends Logging { | ||
|
|
||
| def this(executeHolder: ExecuteHolder) = { | ||
| this(executeHolder.sessionHolder, Some(executeHolder)) | ||
| } | ||
|
|
||
| if (!executeHolderOpt.forall { e => e.sessionHolder == sessionHolder }) { | ||
| throw new IllegalArgumentException("executeHolder does not belong to sessionHolder") | ||
| } | ||
|
|
||
| private[connect] def session: SparkSession = sessionHolder.session | ||
|
|
||
| private[connect] def userId: String = sessionHolder.userId | ||
|
|
||
| private[connect] def sessionId: String = sessionHolder.sessionId | ||
|
|
||
| lazy val executeHolder = executeHolderOpt.getOrElse { | ||
| throw new IllegalArgumentException("executeHolder is not set") | ||
| } | ||
|
|
||
| private lazy val pythonExec = | ||
| sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3")) | ||
|
|
||
|
|
@@ -2461,48 +2476,39 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
|
|
||
| def process( | ||
| command: proto.Command, | ||
| responseObserver: StreamObserver[ExecutePlanResponse], | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { | ||
|
Comment on lines
2477
to
+2479
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Conceptually, this shouldn't be an issue because it's not really a public API. but it's not backwards compatible. This is particularly interesting because you went through some length to fix this problem in the constructor though. @hvanhovell Do we consider the SparkConenct planner to be public? Probably, not correct?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. FWIW, between Spark 3.4 and Spark 3.5, these interfaces also changed.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, I don't think we should. In the end that will hamper our ability to evolve the interface. |
||
| command.getCommandTypeCase match { | ||
| case proto.Command.CommandTypeCase.REGISTER_FUNCTION => | ||
| handleRegisterUserDefinedFunction(command.getRegisterFunction, executeHolder) | ||
| handleRegisterUserDefinedFunction(command.getRegisterFunction) | ||
| case proto.Command.CommandTypeCase.REGISTER_TABLE_FUNCTION => | ||
| handleRegisterUserDefinedTableFunction(command.getRegisterTableFunction, executeHolder) | ||
| handleRegisterUserDefinedTableFunction(command.getRegisterTableFunction) | ||
| case proto.Command.CommandTypeCase.WRITE_OPERATION => | ||
| handleWriteOperation(command.getWriteOperation, executeHolder) | ||
| handleWriteOperation(command.getWriteOperation) | ||
| case proto.Command.CommandTypeCase.CREATE_DATAFRAME_VIEW => | ||
| handleCreateViewCommand(command.getCreateDataframeView, executeHolder) | ||
| handleCreateViewCommand(command.getCreateDataframeView) | ||
| case proto.Command.CommandTypeCase.WRITE_OPERATION_V2 => | ||
| handleWriteOperationV2(command.getWriteOperationV2, executeHolder) | ||
| handleWriteOperationV2(command.getWriteOperationV2) | ||
| case proto.Command.CommandTypeCase.EXTENSION => | ||
| handleCommandPlugin(command.getExtension, executeHolder) | ||
| handleCommandPlugin(command.getExtension) | ||
| case proto.Command.CommandTypeCase.SQL_COMMAND => | ||
| handleSqlCommand(command.getSqlCommand, responseObserver, executeHolder) | ||
| handleSqlCommand(command.getSqlCommand, responseObserver) | ||
| case proto.Command.CommandTypeCase.WRITE_STREAM_OPERATION_START => | ||
| handleWriteStreamOperationStart( | ||
| command.getWriteStreamOperationStart, | ||
| responseObserver, | ||
| executeHolder) | ||
| handleWriteStreamOperationStart(command.getWriteStreamOperationStart, responseObserver) | ||
| case proto.Command.CommandTypeCase.STREAMING_QUERY_COMMAND => | ||
| handleStreamingQueryCommand( | ||
| command.getStreamingQueryCommand, | ||
| responseObserver, | ||
| executeHolder) | ||
| handleStreamingQueryCommand(command.getStreamingQueryCommand, responseObserver) | ||
| case proto.Command.CommandTypeCase.STREAMING_QUERY_MANAGER_COMMAND => | ||
| handleStreamingQueryManagerCommand( | ||
| command.getStreamingQueryManagerCommand, | ||
| responseObserver, | ||
| executeHolder) | ||
| responseObserver) | ||
| case proto.Command.CommandTypeCase.GET_RESOURCES_COMMAND => | ||
| handleGetResourcesCommand(responseObserver, executeHolder) | ||
| handleGetResourcesCommand(responseObserver) | ||
| case _ => throw new UnsupportedOperationException(s"$command not supported.") | ||
| } | ||
| } | ||
|
|
||
| def handleSqlCommand( | ||
| getSqlCommand: SqlCommand, | ||
| responseObserver: StreamObserver[ExecutePlanResponse], | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { | ||
| // Eagerly execute commands of the provided SQL string. | ||
| val args = getSqlCommand.getArgsMap | ||
| val namedArguments = getSqlCommand.getNamedArgumentsMap | ||
|
|
@@ -2600,8 +2606,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
| } | ||
|
|
||
| private def handleRegisterUserDefinedFunction( | ||
| fun: proto.CommonInlineUserDefinedFunction, | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| fun: proto.CommonInlineUserDefinedFunction): Unit = { | ||
| fun.getFunctionCase match { | ||
| case proto.CommonInlineUserDefinedFunction.FunctionCase.PYTHON_UDF => | ||
| handleRegisterPythonUDF(fun) | ||
|
|
@@ -2617,8 +2622,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
| } | ||
|
|
||
| private def handleRegisterUserDefinedTableFunction( | ||
| fun: proto.CommonInlineUserDefinedTableFunction, | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| fun: proto.CommonInlineUserDefinedTableFunction): Unit = { | ||
| fun.getFunctionCase match { | ||
| case proto.CommonInlineUserDefinedTableFunction.FunctionCase.PYTHON_UDTF => | ||
| val function = createPythonUserDefinedTableFunction(fun) | ||
|
|
@@ -2685,7 +2689,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
| session.udf.register(fun.getFunctionName, udf) | ||
| } | ||
|
|
||
| private def handleCommandPlugin(extension: ProtoAny, executeHolder: ExecuteHolder): Unit = { | ||
| private def handleCommandPlugin(extension: ProtoAny): Unit = { | ||
| SparkConnectPluginRegistry.commandRegistry | ||
| // Lazily traverse the collection. | ||
| .view | ||
|
|
@@ -2698,9 +2702,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
| executeHolder.eventsManager.postFinished() | ||
| } | ||
|
|
||
| private def handleCreateViewCommand( | ||
| createView: proto.CreateDataFrameViewCommand, | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| private def handleCreateViewCommand(createView: proto.CreateDataFrameViewCommand): Unit = { | ||
| val viewType = if (createView.getIsGlobal) GlobalTempView else LocalTempView | ||
|
|
||
| val tableIdentifier = | ||
|
|
@@ -2736,9 +2738,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
| * | ||
| * @param writeOperation | ||
| */ | ||
| private def handleWriteOperation( | ||
| writeOperation: proto.WriteOperation, | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| private def handleWriteOperation(writeOperation: proto.WriteOperation): Unit = { | ||
| // Transform the input plan into the logical plan. | ||
| val plan = transformRelation(writeOperation.getInput) | ||
| // And create a Dataset from the plan. | ||
|
|
@@ -2810,9 +2810,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
| * | ||
| * @param writeOperation | ||
| */ | ||
| def handleWriteOperationV2( | ||
| writeOperation: proto.WriteOperationV2, | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| def handleWriteOperationV2(writeOperation: proto.WriteOperationV2): Unit = { | ||
| // Transform the input plan into the logical plan. | ||
| val plan = transformRelation(writeOperation.getInput) | ||
| // And create a Dataset from the plan. | ||
|
|
@@ -2873,8 +2871,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
|
|
||
| def handleWriteStreamOperationStart( | ||
| writeOp: WriteStreamOperationStart, | ||
| responseObserver: StreamObserver[ExecutePlanResponse], | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { | ||
| val plan = transformRelation(writeOp.getInput) | ||
| val tracker = executeHolder.eventsManager.createQueryPlanningTracker | ||
| val dataset = Dataset.ofRows(session, plan, tracker) | ||
|
|
@@ -2999,8 +2996,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
|
|
||
| def handleStreamingQueryCommand( | ||
| command: StreamingQueryCommand, | ||
| responseObserver: StreamObserver[ExecutePlanResponse], | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { | ||
|
|
||
| val id = command.getQueryId.getId | ||
| val runId = command.getQueryId.getRunId | ||
|
|
@@ -3177,8 +3173,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
|
|
||
| def handleStreamingQueryManagerCommand( | ||
| command: StreamingQueryManagerCommand, | ||
| responseObserver: StreamObserver[ExecutePlanResponse], | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { | ||
| val respBuilder = StreamingQueryManagerCommandResult.newBuilder() | ||
|
|
||
| command.getCommandCase match { | ||
|
|
@@ -3257,8 +3252,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { | |
| } | ||
|
|
||
| def handleGetResourcesCommand( | ||
| responseObserver: StreamObserver[proto.ExecutePlanResponse], | ||
| executeHolder: ExecuteHolder): Unit = { | ||
| responseObserver: StreamObserver[proto.ExecutePlanResponse]): Unit = { | ||
| executeHolder.eventsManager.postFinished() | ||
| responseObserver.onNext( | ||
| proto.ExecutePlanResponse | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why not just pass in the ExecuteHolder, and create a getter for SessionHolder? IMO it is a bit weird to create this structure for just a single unit test.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it's used in AnalyzePlan that doesn't have an ExecuteHolder
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok, makes sense.