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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.
+  private var responseSender: Option[ExecutePlanResponseSender] = None

Review Comment:
   Do you want to document that the specific instance of the 
`ExecutePlanResponseSender` might change because it is the connection to the 
actual gprc response handler?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.

Review Comment:
   ```suggestion
     // Sender to notify of available responses.
   ```



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1

Review Comment:
   for consistency instead of being called index, should this then be 
`lastIndex`? Since this makes clearer what the intend is? Otherwise, readers 
might believe that this is a zero based offset into the `responses`?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.
+  private var responseSender: Option[ExecutePlanResponseSender] = None
+
+  def onNext(r: ExecutePlanResponse): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onNext can't be called after 
stream completed")
+    }
+    index += 1
+    responses += CachedExecutePlanResponse(r, index)
+    logDebug(s"Saved response with index=$index")
+    notifyAll()
+  }
+
+  def onError(t: Throwable): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onError can't be called after 
stream completed")
+    }
+    error = Some(t)
+    lastIndex = Some(index) // no responses to be send after error.
+    logDebug(s"Error. Last stream index is $index.")
+    notifyAll()
+  }
+
+  def onCompleted(): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onCompleted can't be called 
after stream completed")
+    }
+    lastIndex = Some(index)
+    logDebug(s"Completed. Last stream index is $index.")
+    notifyAll()
+  }
+
+  /** Set a new response sender. */
+  def setExecutePlanResponseSender(newSender: ExecutePlanResponseSender): Unit 
= synchronized {
+    // detach the current sender before attaching new one
+    // this.synchronized() needs to be held while detaching a sender, and the 
detached sender
+    // needs to be notified with notifyAll() afterwards.
+    responseSender.foreach(_.detach())
+    responseSender = Some(newSender)
+    notifyAll()
+  }
+
+  /** Remove cached responses until index */
+  def removeUntilIndex(index: Long): Unit = synchronized {

Review Comment:
   should this be public to the consumer? Do you want to make it a two step 
process for removing and getting responses?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.
+  private var responseSender: Option[ExecutePlanResponseSender] = None
+
+  def onNext(r: ExecutePlanResponse): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onNext can't be called after 
stream completed")
+    }
+    index += 1
+    responses += CachedExecutePlanResponse(r, index)
+    logDebug(s"Saved response with index=$index")
+    notifyAll()
+  }
+
+  def onError(t: Throwable): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onError can't be called after 
stream completed")
+    }
+    error = Some(t)
+    lastIndex = Some(index) // no responses to be send after error.
+    logDebug(s"Error. Last stream index is $index.")
+    notifyAll()
+  }
+
+  def onCompleted(): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onCompleted can't be called 
after stream completed")
+    }
+    lastIndex = Some(index)
+    logDebug(s"Completed. Last stream index is $index.")
+    notifyAll()
+  }
+
+  /** Set a new response sender. */
+  def setExecutePlanResponseSender(newSender: ExecutePlanResponseSender): Unit 
= synchronized {
+    // detach the current sender before attaching new one
+    // this.synchronized() needs to be held while detaching a sender, and the 
detached sender
+    // needs to be notified with notifyAll() afterwards.
+    responseSender.foreach(_.detach())
+    responseSender = Some(newSender)
+    notifyAll()
+  }
+
+  /** Remove cached responses until index */
+  def removeUntilIndex(index: Long): Unit = synchronized {
+    while (responses.nonEmpty && responses(0).index <= index) {
+      responses.remove(0)
+    }
+    logDebug(s"Removed saved responses until index $index.")
+  }
+
+  /** Get response with a given index in the stream, if set. */
+  def getResponse(index: Long): Option[CachedExecutePlanResponse] = 
synchronized {
+    if (responses.nonEmpty && (index - responses(0).index).toInt < 
responses.size) {

Review Comment:
   This is a bit weird, you use the responses size to allow concurrent fetching 
while the query is not done, but keep track of the overall lastIndex as well.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.
+  private var responseSender: Option[ExecutePlanResponseSender] = None
+
+  def onNext(r: ExecutePlanResponse): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onNext can't be called after 
stream completed")
+    }
+    index += 1
+    responses += CachedExecutePlanResponse(r, index)
+    logDebug(s"Saved response with index=$index")
+    notifyAll()
+  }
+
+  def onError(t: Throwable): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onError can't be called after 
stream completed")
+    }
+    error = Some(t)
+    lastIndex = Some(index) // no responses to be send after error.
+    logDebug(s"Error. Last stream index is $index.")
+    notifyAll()
+  }
+
+  def onCompleted(): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onCompleted can't be called 
after stream completed")
+    }
+    lastIndex = Some(index)
+    logDebug(s"Completed. Last stream index is $index.")
+    notifyAll()
+  }
+
+  /** Set a new response sender. */
+  def setExecutePlanResponseSender(newSender: ExecutePlanResponseSender): Unit 
= synchronized {
+    // detach the current sender before attaching new one
+    // this.synchronized() needs to be held while detaching a sender, and the 
detached sender
+    // needs to be notified with notifyAll() afterwards.
+    responseSender.foreach(_.detach())
+    responseSender = Some(newSender)
+    notifyAll()
+  }
+
+  /** Remove cached responses until index */
+  def removeUntilIndex(index: Long): Unit = synchronized {
+    while (responses.nonEmpty && responses(0).index <= index) {
+      responses.remove(0)
+    }
+    logDebug(s"Removed saved responses until index $index.")
+  }
+
+  /** Get response with a given index in the stream, if set. */
+  def getResponse(index: Long): Option[CachedExecutePlanResponse] = 
synchronized {
+    if (responses.nonEmpty && (index - responses(0).index).toInt < 
responses.size) {
+      // Note: index access in ListBuffer is linear; we assume here the buffer 
is not too long.
+      val ret = responses((index - responses(0).index).toInt)
+      assert(ret.index == index)
+      Some(ret)
+    } else {
+      None
+    }
+  }
+
+  /** Get the stream error, if set.  */
+  def getError(): Option[Throwable] = synchronized {
+    error
+  }
+
+  /** If the stream is finished, the index of the last response, otherwise 
unset. */
+  def getLastIndex(): Option[Long] = synchronized {
+    lastIndex
+  }
+
+  def completed(): Boolean = synchronized {
+    lastIndex.isDefined

Review Comment:
   why not make lastIndex a flag `completed` instead



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.
+  private var responseSender: Option[ExecutePlanResponseSender] = None
+
+  def onNext(r: ExecutePlanResponse): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onNext can't be called after 
stream completed")
+    }
+    index += 1
+    responses += CachedExecutePlanResponse(r, index)
+    logDebug(s"Saved response with index=$index")
+    notifyAll()
+  }
+
+  def onError(t: Throwable): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onError can't be called after 
stream completed")
+    }
+    error = Some(t)
+    lastIndex = Some(index) // no responses to be send after error.
+    logDebug(s"Error. Last stream index is $index.")
+    notifyAll()
+  }
+
+  def onCompleted(): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onCompleted can't be called 
after stream completed")
+    }
+    lastIndex = Some(index)
+    logDebug(s"Completed. Last stream index is $index.")
+    notifyAll()
+  }
+
+  /** Set a new response sender. */
+  def setExecutePlanResponseSender(newSender: ExecutePlanResponseSender): Unit 
= synchronized {
+    // detach the current sender before attaching new one
+    // this.synchronized() needs to be held while detaching a sender, and the 
detached sender
+    // needs to be notified with notifyAll() afterwards.
+    responseSender.foreach(_.detach())
+    responseSender = Some(newSender)
+    notifyAll()
+  }
+
+  /** Remove cached responses until index */
+  def removeUntilIndex(index: Long): Unit = synchronized {
+    while (responses.nonEmpty && responses(0).index <= index) {
+      responses.remove(0)
+    }
+    logDebug(s"Removed saved responses until index $index.")
+  }
+
+  /** Get response with a given index in the stream, if set. */
+  def getResponse(index: Long): Option[CachedExecutePlanResponse] = 
synchronized {
+    if (responses.nonEmpty && (index - responses(0).index).toInt < 
responses.size) {
+      // Note: index access in ListBuffer is linear; we assume here the buffer 
is not too long.
+      val ret = responses((index - responses(0).index).toInt)

Review Comment:
   passing in a negative index will probably crash and burn



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteRunner.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.util.control.NonFatal
+
+import com.google.protobuf.Message
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.spark.SparkSQLException
+import org.apache.spark.connect.proto
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.connect.common.ProtoUtils
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.connect.service.ExecuteHolder
+import org.apache.spark.sql.connect.utils.ErrorUtils
+import org.apache.spark.util.Utils
+
+private[connect] class ExecuteRunner(executeHolder: ExecuteHolder) extends 
Logging {
+
+  private var executionThread: Thread = new ExecutionThread()
+
+  private var interrupted: Boolean = false
+
+  def start(): 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.start()
+    }
+  }
+
+  private def execute(): Unit = {
+    // Outer execute
+    try {
+      try {
+        execute()

Review Comment:
   ```suggestion
           executeInternal()
   ```
   ??



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.

Review Comment:
   I think lastInedx should be renamed to something like `finalResponseSize` to 
indicate that it's used as as size value for the response instead of an actual 
index.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1

Review Comment:
   ```suggestion
     private var lastIndexSent: Long = 0 // first response will have index 1
   ```



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseSender.scala:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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 io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * ExecutePlanResponseSender sends responses to the GRPC stream.
+ * It runs on the RPC thread, and gets notified by ExecutePlanResponseObserver 
about available
+ * responses.
+ * It notifies the ExecutePlanResponseObserver back about cached responses 
that can be removed
+ * after being sent out.
+ * @param responseObserver the GRPC request StreamObserver
+ */
+private[connect] class ExecutePlanResponseSender(
+  grpcObserver: StreamObserver[ExecutePlanResponse]) extends Logging {
+
+  private var detached = false
+
+  /** Detach this sender from executionObserver.
+   *  Called only from executionObserver that this sender is attached to.
+   *  executionObserver holds lock, and needs to notify after this call. */
+  def detach(): Unit = {
+    if (detached == true) {
+      throw new IllegalStateException("ExecutePlanResponseSender already 
detached!")
+    }
+    detached = true
+  }
+
+  /**
+   * Receive responses from executionObserver and send them to grpcObserver.
+   * @param lastSentIndex Start sending the stream from response after this.
+   * @return true if the execution was detached before stream completed.
+   *         The caller needs to finish the grpcObserver stream
+   *         false if stream was finished. In this case, grpcObserver stream 
is already completed.
+   */
+  def run(executionObserver: ExecutePlanResponseObserver, lastSentIndex: 
Long): Boolean = {

Review Comment:
   Do I read this method correctly that this will try to push all messages to 
the grpc observer?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.
+  private var responseSender: Option[ExecutePlanResponseSender] = None
+
+  def onNext(r: ExecutePlanResponse): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onNext can't be called after 
stream completed")
+    }
+    index += 1
+    responses += CachedExecutePlanResponse(r, index)
+    logDebug(s"Saved response with index=$index")
+    notifyAll()
+  }
+
+  def onError(t: Throwable): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onError can't be called after 
stream completed")
+    }
+    error = Some(t)
+    lastIndex = Some(index) // no responses to be send after error.
+    logDebug(s"Error. Last stream index is $index.")
+    notifyAll()
+  }
+
+  def onCompleted(): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onCompleted can't be called 
after stream completed")
+    }
+    lastIndex = Some(index)
+    logDebug(s"Completed. Last stream index is $index.")
+    notifyAll()
+  }
+
+  /** Set a new response sender. */
+  def setExecutePlanResponseSender(newSender: ExecutePlanResponseSender): Unit 
= synchronized {
+    // detach the current sender before attaching new one
+    // this.synchronized() needs to be held while detaching a sender, and the 
detached sender
+    // needs to be notified with notifyAll() afterwards.
+    responseSender.foreach(_.detach())
+    responseSender = Some(newSender)
+    notifyAll()
+  }
+
+  /** Remove cached responses until index */
+  def removeUntilIndex(index: Long): Unit = synchronized {
+    while (responses.nonEmpty && responses(0).index <= index) {
+      responses.remove(0)
+    }
+    logDebug(s"Removed saved responses until index $index.")
+  }
+
+  /** Get response with a given index in the stream, if set. */
+  def getResponse(index: Long): Option[CachedExecutePlanResponse] = 
synchronized {
+    if (responses.nonEmpty && (index - responses(0).index).toInt < 
responses.size) {
+      // Note: index access in ListBuffer is linear; we assume here the buffer 
is not too long.
+      val ret = responses((index - responses(0).index).toInt)
+      assert(ret.index == index)
+      Some(ret)
+    } else {
+      None

Review Comment:
   should it throw or None?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseObserver.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.mutable.ListBuffer
+
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * Container for ExecutePlanResponses responses.
+ *
+ * This StreamObserver is running on the execution thread and saves the 
responses,
+ * it notifies the ExecutePlanResponseSender about available responses.
+ *
+ * @param responseObserver
+ */
+private[connect] class ExecutePlanResponseObserver()
+  extends StreamObserver[ExecutePlanResponse]
+  with Logging {
+
+  // Cached stream state.
+  private val responses = new ListBuffer[CachedExecutePlanResponse]()
+  private var error: Option[Throwable] = None
+  private var lastIndex: Option[Long] = None // index of last response before 
completed.
+  private var index: Long = 0 // first response will have index 1
+
+  // sender to notify of available responses.
+  private var responseSender: Option[ExecutePlanResponseSender] = None
+
+  def onNext(r: ExecutePlanResponse): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onNext can't be called after 
stream completed")
+    }
+    index += 1
+    responses += CachedExecutePlanResponse(r, index)
+    logDebug(s"Saved response with index=$index")
+    notifyAll()
+  }
+
+  def onError(t: Throwable): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onError can't be called after 
stream completed")
+    }
+    error = Some(t)
+    lastIndex = Some(index) // no responses to be send after error.
+    logDebug(s"Error. Last stream index is $index.")
+    notifyAll()
+  }
+
+  def onCompleted(): Unit = synchronized {
+    if (lastIndex.nonEmpty) {
+      throw new IllegalStateException("Stream onCompleted can't be called 
after stream completed")
+    }
+    lastIndex = Some(index)
+    logDebug(s"Completed. Last stream index is $index.")
+    notifyAll()
+  }
+
+  /** Set a new response sender. */
+  def setExecutePlanResponseSender(newSender: ExecutePlanResponseSender): Unit 
= synchronized {
+    // detach the current sender before attaching new one
+    // this.synchronized() needs to be held while detaching a sender, and the 
detached sender
+    // needs to be notified with notifyAll() afterwards.
+    responseSender.foreach(_.detach())
+    responseSender = Some(newSender)
+    notifyAll()
+  }
+
+  /** Remove cached responses until index */
+  def removeUntilIndex(index: Long): Unit = synchronized {
+    while (responses.nonEmpty && responses(0).index <= index) {
+      responses.remove(0)
+    }
+    logDebug(s"Removed saved responses until index $index.")
+  }
+
+  /** Get response with a given index in the stream, if set. */
+  def getResponse(index: Long): Option[CachedExecutePlanResponse] = 
synchronized {
+    if (responses.nonEmpty && (index - responses(0).index).toInt < 
responses.size) {
+      // Note: index access in ListBuffer is linear; we assume here the buffer 
is not too long.
+      val ret = responses((index - responses(0).index).toInt)
+      assert(ret.index == index)
+      Some(ret)
+    } else {
+      None
+    }
+  }
+
+  /** Get the stream error, if set.  */
+  def getError(): Option[Throwable] = synchronized {
+    error
+  }
+
+  /** If the stream is finished, the index of the last response, otherwise 
unset. */
+  def getLastIndex(): Option[Long] = synchronized {
+    lastIndex

Review Comment:
   ... and then you can return here `if completed return responses.size else 
None`



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecutePlanResponseSender.scala:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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 io.grpc.stub.StreamObserver
+
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.internal.Logging
+
+/**
+ * ExecutePlanResponseSender sends responses to the GRPC stream.
+ * It runs on the RPC thread, and gets notified by ExecutePlanResponseObserver 
about available
+ * responses.
+ * It notifies the ExecutePlanResponseObserver back about cached responses 
that can be removed
+ * after being sent out.
+ * @param responseObserver the GRPC request StreamObserver
+ */
+private[connect] class ExecutePlanResponseSender(
+  grpcObserver: StreamObserver[ExecutePlanResponse]) extends Logging {
+
+  private var detached = false
+
+  /** Detach this sender from executionObserver.
+   *  Called only from executionObserver that this sender is attached to.
+   *  executionObserver holds lock, and needs to notify after this call. */
+  def detach(): Unit = {
+    if (detached == true) {
+      throw new IllegalStateException("ExecutePlanResponseSender already 
detached!")
+    }
+    detached = true
+  }
+
+  /**
+   * Receive responses from executionObserver and send them to grpcObserver.
+   * @param lastSentIndex Start sending the stream from response after this.

Review Comment:
   ```suggestion
      * @param lastSentIndex Start sending the stream from response after this.
                             The lastSentIndex indicates the last response seen 
by
                             the consumer. 
   ```



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