grundprinzip commented on code in PR #41527:
URL: https://github.com/apache/spark/pull/41527#discussion_r1245421892
##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -96,31 +97,33 @@ class SparkConnectStreamHandler(responseObserver:
StreamObserver[ExecutePlanResp
private def handlePlan(sessionHolder: SessionHolder, request:
ExecutePlanRequest): Unit = {
// Extract the plan from the request and convert it to a logical plan
- val planner = new SparkConnectPlanner(sessionHolder)
+ val planner = new SparkConnectPlanner(sessionHolder, Some(this))
val dataframe =
Dataset.ofRows(sessionHolder.session,
planner.transformRelation(request.getPlan.getRoot))
- responseObserver.onNext(
+ sendResponse(
SparkConnectStreamHandler.sendSchemaToResponse(request.getSessionId,
dataframe.schema))
processAsArrowBatches(request.getSessionId, dataframe, responseObserver)
- responseObserver.onNext(
- SparkConnectStreamHandler.createMetricsResponse(request.getSessionId,
dataframe))
+
sendResponse(SparkConnectStreamHandler.createMetricsResponse(request.getSessionId,
dataframe))
if (dataframe.queryExecution.observedMetrics.nonEmpty) {
- responseObserver.onNext(
+ sendResponse(
SparkConnectStreamHandler.sendObservedMetricsToResponse(request.getSessionId,
dataframe))
}
responseObserver.onCompleted()
}
private def handleCommand(sessionHolder: SessionHolder, request:
ExecutePlanRequest): Unit = {
val command = request.getPlan.getCommand
- val planner = new SparkConnectPlanner(sessionHolder)
+ val planner = new SparkConnectPlanner(sessionHolder, Some(this))
planner.process(
command = command,
userId = request.getUserContext.getUserId,
- sessionId = request.getSessionId,
- responseObserver = responseObserver)
+ sessionId = request.getSessionId)
responseObserver.onCompleted()
}
+
+ def sendResponse(response: ExecutePlanResponse): Unit = {
Review Comment:
should we make this `private[connect]`?
##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1061,7 +1057,7 @@ class SparkConnectPlanner(val sessionHolder:
SessionHolder) extends Logging {
}
}
- private def transformDataType(t: proto.DataType): DataType = {
Review Comment:
why remove the `private`? This should at least be package private.
##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -84,13 +69,24 @@ 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,
+ responseHandlerOpt: Option[SparkConnectResponseHandler[_]] = None)
Review Comment:
Is there a valid use-case where this can be empty? Personally, I don't think
that this should ever be none. I understand for testing purposes one might
inject a mock, but otherwise the planner cannot exist without the
responsehandler
--
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]