anishshri-db commented on code in PR #47133:
URL: https://github.com/apache/spark/pull/47133#discussion_r1680047393


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.execution.python
+
+import java.io.{DataInputStream, DataOutputStream, EOFException}
+import java.nio.channels.Channels
+
+import scala.collection.mutable
+
+import com.google.protobuf.ByteString
+import jnr.unixsocket.UnixServerSocketChannel
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Encoder, Encoders, Row}
+import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, 
StatefulProcessorHandleImpl, StatefulProcessorHandleState}
+import 
org.apache.spark.sql.execution.streaming.state.StateMessage.{HandleState, 
ImplicitGroupingKeyRequest, StatefulProcessorCall, StateRequest, StateResponse, 
StateVariableRequest, ValueStateCall}
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types.{BooleanType, DataType, DoubleType, 
FloatType, IntegerType, LongType, StructType}
+
+/**
+ * This class is used to handle the state requests from the Python side.
+ */
+class TransformWithStateInPandasStateServer(
+    private val serverChannel: UnixServerSocketChannel,
+    private val statefulProcessorHandle: StatefulProcessorHandleImpl,
+    private val groupingKeySchema: StructType)
+  extends Runnable
+  with Logging{
+
+  private var inputStream: DataInputStream = _
+  private var outputStream: DataOutputStream = _
+
+  private val valueStates = mutable.HashMap[String, ValueState[Any]]()
+
+  def run(): Unit = {
+    logWarning(s"Waiting for connection from Python worker")
+    val channel = serverChannel.accept()
+    logWarning(s"listening on channel - ${channel.getLocalAddress}")
+
+    inputStream = new DataInputStream(
+      Channels.newInputStream(channel))
+    outputStream = new DataOutputStream(
+      Channels.newOutputStream(channel)
+    )
+
+    while (channel.isConnected &&
+      statefulProcessorHandle.getHandleState != 
StatefulProcessorHandleState.CLOSED) {
+
+      try {
+        logWarning(s"reading the version")
+        val version = inputStream.readInt()
+
+        if (version != -1) {
+          logWarning(s"version = ${version}")
+          assert(version == 0)
+          val messageLen = inputStream.readInt()
+          logWarning(s"parsing a message of ${messageLen} bytes")
+
+          val messageBytes = new Array[Byte](messageLen)
+          inputStream.read(messageBytes)
+          logWarning(s"read bytes = ${messageBytes.mkString("Array(", ", ", 
")")}")
+
+          val message = 
StateRequest.parseFrom(ByteString.copyFrom(messageBytes))
+
+          logWarning(s"read message = $message")
+          handleRequest(message)
+          logWarning(s"flush output stream")
+
+          outputStream.flush()
+        }
+      } catch {
+        case _: EOFException =>
+          logWarning(s"No more data to read from the socket")
+          
statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CLOSED)
+          return
+        case e: Exception =>
+          logWarning(s"Error reading message: ${e.getMessage}")
+          sendResponse(1, e.getMessage)
+          outputStream.flush()
+          
statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CLOSED)
+          return
+      }
+    }
+    logWarning(s"done from the state server thread")
+  }
+
+  private def handleRequest(message: StateRequest): Unit = {
+    if (message.getMethodCase == 
StateRequest.MethodCase.STATEFULPROCESSORCALL) {
+      val statefulProcessorHandleRequest = message.getStatefulProcessorCall
+      if (statefulProcessorHandleRequest.getMethodCase ==
+        StatefulProcessorCall.MethodCase.SETHANDLESTATE) {
+        val requestedState = 
statefulProcessorHandleRequest.getSetHandleState.getState
+        requestedState match {
+          case HandleState.CREATED =>
+            logWarning(s"set handle state to Created")
+            
statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CREATED)
+          case HandleState.INITIALIZED =>
+            logWarning(s"set handle state to Initialized")
+            
statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.INITIALIZED)
+          case HandleState.CLOSED =>
+            logWarning(s"set handle state to Closed")
+            
statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CLOSED)
+          case _ =>
+        }
+        sendResponse(0)
+      } else if (statefulProcessorHandleRequest.getMethodCase ==

Review Comment:
   This method is becoming huge. Can we split each case/msg into a separate 
function maybe ?



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

Reply via email to