grundprinzip commented on code in PR #41315:
URL: https://github.com/apache/spark/pull/41315#discussion_r1206882959


##########
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()
+        }
+      }

Review Comment:
   Is there a particular reason for the nesting? Why not create an 
ExecutionThread that inherits from Thread instead of creating a thread that 
keeps a reference to the outside that is called from within?



##########
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()

Review Comment:
   does this assume that the execution thread was interrupted?



##########
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 = {

Review Comment:
   ```suggestion
     def interrupt(): Unit = synchronized {
   ```
   might save you an indent.



##########
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()

Review Comment:
   you have too many execute like things here.
   
   You have 
   
   ```
   def run()
      startExecute()
         execute()
   ```
   
   I'm wondering if this can be named better. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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:
   should this be package private?



##########
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()

Review Comment:
   nit: `runInternal`? Just from the naming start feals weird because you're 
already in run?



##########
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() {

Review Comment:
   Is there a reason for using `this.`?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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

Review Comment:
   ```suggestion
    * Handle ExecutePlanRequest where the operation to handle is of `Plan` type
   ```



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala:
##########
@@ -33,26 +32,26 @@ import org.apache.spark.sql.SparkSession
 case class SessionHolder(userId: String, sessionId: String, session: 
SparkSession)
     extends Logging {
 
-  val executePlanOperations: ConcurrentMap[String, ExecutePlanHolder] =
-    new ConcurrentHashMap[String, ExecutePlanHolder]()
+  val executions: ConcurrentMap[String, ExecutionHolder] =
+    new ConcurrentHashMap[String, ExecutionHolder]()
 
-  private[connect] def createExecutePlanHolder(
-      request: proto.ExecutePlanRequest): ExecutePlanHolder = {
+  private[connect] def createExecutionHolder(): ExecutionHolder = {
 
     val operationId = UUID.randomUUID().toString
-    val executePlanHolder = ExecutePlanHolder(operationId, this, request)
-    assert(executePlanOperations.putIfAbsent(operationId, executePlanHolder) 
== null)
+    val executePlanHolder = ExecutionHolder(operationId, this)
+    assert(executions.putIfAbsent(operationId, executePlanHolder) == null)

Review Comment:
   Aren't we removing all of the asserts from the actual code? Should we then 
just mark it a regular exception handling code? I'm a huge fan of invariant 
assertions though :(



-- 
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]

Reply via email to