You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "srowen (via GitHub)" <gi...@apache.org> on 2023/02/14 18:39:18 UTC

[GitHub] [spark] srowen commented on a diff in pull request #39985: [SPARK-42412][WIP] Initial prototype implementation of PySpark ML via Spark connect

srowen commented on code in PR #39985:
URL: https://github.com/apache/spark/pull/39985#discussion_r1106181670


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/MLUtils.scala:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.planner
+
+import org.apache.spark.ml.param.{Param, Params}
+
+object SparkConnectUtils {
+
+  private def castParamValue(paramType: Class[_], paramValue: Object): Object = {
+    if (paramType == classOf[Int]) {

Review Comment:
   Probably the answer is no, but can't this be done with `match`?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/MLUtils.scala:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.planner
+
+import org.apache.spark.ml.param.{Param, Params}
+
+object SparkConnectUtils {
+
+  private def castParamValue(paramType: Class[_], paramValue: Object): Object = {
+    if (paramType == classOf[Int]) {
+      java.lang.Integer.valueOf(paramValue.asInstanceOf[java.lang.Number].intValue())
+    } else if (paramType == classOf[Long]) {
+      java.lang.Long.valueOf(paramValue.asInstanceOf[java.lang.Number].longValue())
+    } else if (paramType == classOf[Float]) {
+      java.lang.Float.valueOf(paramValue.asInstanceOf[java.lang.Number].floatValue())
+    } else if (paramType == classOf[Double]) {
+      java.lang.Double.valueOf(paramValue.asInstanceOf[java.lang.Number].doubleValue())

Review Comment:
   How about Byte? we probably need Array[Byte] at least



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/MLUtils.scala:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.planner
+
+import org.apache.spark.ml.param.{Param, Params}
+
+object SparkConnectUtils {
+
+  private def castParamValue(paramType: Class[_], paramValue: Object): Object = {
+    if (paramType == classOf[Int]) {
+      java.lang.Integer.valueOf(paramValue.asInstanceOf[java.lang.Number].intValue())
+    } else if (paramType == classOf[Long]) {
+      java.lang.Long.valueOf(paramValue.asInstanceOf[java.lang.Number].longValue())
+    } else if (paramType == classOf[Float]) {
+      java.lang.Float.valueOf(paramValue.asInstanceOf[java.lang.Number].floatValue())
+    } else if (paramType == classOf[Double]) {
+      java.lang.Double.valueOf(paramValue.asInstanceOf[java.lang.Number].doubleValue())
+    } else if (paramType == classOf[Array[Int]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Int], _).asInstanceOf[java.lang.Integer])

Review Comment:
   Nit, why not one map() statement doing both?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/MLUtils.scala:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.planner
+
+import org.apache.spark.ml.param.{Param, Params}
+
+object SparkConnectUtils {
+
+  private def castParamValue(paramType: Class[_], paramValue: Object): Object = {
+    if (paramType == classOf[Int]) {
+      java.lang.Integer.valueOf(paramValue.asInstanceOf[java.lang.Number].intValue())
+    } else if (paramType == classOf[Long]) {
+      java.lang.Long.valueOf(paramValue.asInstanceOf[java.lang.Number].longValue())
+    } else if (paramType == classOf[Float]) {
+      java.lang.Float.valueOf(paramValue.asInstanceOf[java.lang.Number].floatValue())
+    } else if (paramType == classOf[Double]) {
+      java.lang.Double.valueOf(paramValue.asInstanceOf[java.lang.Number].doubleValue())
+    } else if (paramType == classOf[Array[Int]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Int], _).asInstanceOf[java.lang.Integer])
+        .map(_.intValue()).toArray
+    } else if (paramType == classOf[Array[Long]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Long], _).asInstanceOf[java.lang.Long])
+        .map(_.longValue()).toArray
+    } else if (paramType == classOf[Array[Float]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Float], _).asInstanceOf[java.lang.Float])
+        .map(_.floatValue()).toArray
+    } else if (paramType == classOf[Array[Double]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Double], _).asInstanceOf[java.lang.Double])
+        .map(_.doubleValue()).toArray
+    } else if (paramType == classOf[Array[Boolean]]) {
+      paramValue.asInstanceOf[List[java.lang.Boolean]].map(_.booleanValue()).toArray
+    } else if (paramType == classOf[Array[String]]) {
+      paramValue.asInstanceOf[List[String]].toArray
+    } else {
+      throw new RuntimeException()

Review Comment:
   At least here say the requested type, and the value



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkMLPlanner.scala:
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.planner
+
+import io.grpc.stub.StreamObserver
+import java.lang
+import scala.reflect.runtime.universe
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{ExecutePlanRequest, ExecutePlanResponse, RemoteCall}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.connect.service.{SessionHolder}
+import org.apache.spark.util.Utils
+
+object SparkMLPlanner {
+
+  val runtimeMirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+
+  def buildRemoteCallResponse(
+      request: ExecutePlanRequest,
+      responseObserver: StreamObserver[ExecutePlanResponse],
+      returnValueOpt: Option[RemoteCall.ArgValue]
+  ): Unit = {
+    val response = proto.ExecutePlanResponse.newBuilder
+      .setClientId(request.getClientId)
+
+    returnValueOpt.foreach(response.setRemoteCallReturnValue(_))

Review Comment:
   Total nit, you can omit (_) usually



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/MLUtils.scala:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.planner
+
+import org.apache.spark.ml.param.{Param, Params}
+
+object SparkConnectUtils {
+
+  private def castParamValue(paramType: Class[_], paramValue: Object): Object = {
+    if (paramType == classOf[Int]) {
+      java.lang.Integer.valueOf(paramValue.asInstanceOf[java.lang.Number].intValue())
+    } else if (paramType == classOf[Long]) {
+      java.lang.Long.valueOf(paramValue.asInstanceOf[java.lang.Number].longValue())
+    } else if (paramType == classOf[Float]) {
+      java.lang.Float.valueOf(paramValue.asInstanceOf[java.lang.Number].floatValue())
+    } else if (paramType == classOf[Double]) {
+      java.lang.Double.valueOf(paramValue.asInstanceOf[java.lang.Number].doubleValue())
+    } else if (paramType == classOf[Array[Int]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Int], _).asInstanceOf[java.lang.Integer])
+        .map(_.intValue()).toArray
+    } else if (paramType == classOf[Array[Long]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Long], _).asInstanceOf[java.lang.Long])
+        .map(_.longValue()).toArray
+    } else if (paramType == classOf[Array[Float]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Float], _).asInstanceOf[java.lang.Float])
+        .map(_.floatValue()).toArray
+    } else if (paramType == classOf[Array[Double]]) {
+      paramValue.asInstanceOf[List[Object]]
+        .map(castParamValue(classOf[Double], _).asInstanceOf[java.lang.Double])
+        .map(_.doubleValue()).toArray
+    } else if (paramType == classOf[Array[Boolean]]) {
+      paramValue.asInstanceOf[List[java.lang.Boolean]].map(_.booleanValue()).toArray
+    } else if (paramType == classOf[Array[String]]) {
+      paramValue.asInstanceOf[List[String]].toArray
+    } else {
+      throw new RuntimeException()
+    }
+  }
+
+  private def makeParamPair(
+                             instance: Params, paramName: String, paramValue: Object

Review Comment:
   Nit, wrong wrapping



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkMLPlanner.scala:
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.planner
+
+import io.grpc.stub.StreamObserver
+import java.lang
+import scala.reflect.runtime.universe
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{ExecutePlanRequest, ExecutePlanResponse, RemoteCall}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.connect.service.{SessionHolder}
+import org.apache.spark.util.Utils
+
+object SparkMLPlanner {
+
+  val runtimeMirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+
+  def buildRemoteCallResponse(
+      request: ExecutePlanRequest,
+      responseObserver: StreamObserver[ExecutePlanResponse],
+      returnValueOpt: Option[RemoteCall.ArgValue]
+  ): Unit = {
+    val response = proto.ExecutePlanResponse.newBuilder
+      .setClientId(request.getClientId)
+
+    returnValueOpt.foreach(response.setRemoteCallReturnValue(_))
+
+    responseObserver.onNext(response.build())
+    responseObserver.onCompleted()
+  }
+
+  def loadInputRelation(sessionHolder: SessionHolder, inputRelation: proto.Relation): DataFrame = {
+    val relationalPlanner = new SparkConnectPlanner(sessionHolder)
+    val plan = relationalPlanner.transformRelation(inputRelation)
+    Dataset.ofRows(sessionHolder.session, plan)
+  }
+
+  /*
+  def createStage(uid: String, className: String): PipelineStage = {
+    val clazz = Utils.classForName(className)
+    val ctorOpt = clazz.getConstructors.find { ctor =>
+      // Find the constructor with signature `this(uid: String)`
+      ctor.getParameterCount == 1 && ctor.getParameterTypes()(0).eq(classOf[String])
+    }
+    if (ctorOpt.isEmpty) {
+      throw new RuntimeException(
+        s"Could not find 'this(uid: String)' constructor for class $className"
+      )
+    }
+
+    ctorOpt.get.newInstance(uid).asInstanceOf[PipelineStage]
+  }
+
+  def extractParamPair(
+     instance: Params,
+     paramName: String,
+     protoValue: MlCommand.ParamValue
+  ): (Param[Any], Any) = {
+    import MlCommand.ParamValue.ValueCase
+
+    val param = instance.getParam(paramName)
+    val valueType = param.paramValueClassTag.runtimeClass
+    val extractedValue = protoValue.getValueCase match {
+      case ValueCase.INT_VAL =>
+        assert(valueType == classOf[Int])
+        protoValue.getIntVal
+
+      case ValueCase.LONG_VAL =>
+        if (valueType == classOf[Long]) {
+          protoValue.getLongVal
+        } else if (valueType == classOf[Int]) {
+          protoValue.getLongVal.toInt
+        } else {
+          throw new java.lang.AssertionError()
+        }
+
+      case ValueCase.FLOAT_VAL =>
+        assert(valueType == classOf[Float])
+        protoValue.getFloatVal
+
+      case ValueCase.DOUBLE_VAL =>
+        if (valueType == classOf[Double]) {
+          protoValue.getDoubleVal
+        } else if (valueType == classOf[Float]) {
+          protoValue.getDoubleVal.toFloat
+        } else {
+          throw new java.lang.AssertionError()
+        }
+
+      case ValueCase.BOOL_VAL =>
+        assert(valueType == classOf[Boolean])
+        protoValue.getBoolVal
+
+      case ValueCase.STR_VAL =>
+        assert(valueType == classOf[String])
+        protoValue.getStrVal
+
+      case ValueCase.INT_ARRAY =>
+        assert(valueType == classOf[Array[Int]])
+        protoValue.getIntArray.getElementList.stream().mapToInt(_.toInt).toArray
+
+      case ValueCase.LONG_ARRAY =>
+        val elemList = protoValue.getLongArray.getElementList
+        if (valueType == classOf[Array[Long]]) {
+          elemList.stream().mapToLong(_.toLong).toArray
+        } else if (valueType == classOf[Array[Int]]) {
+          elemList.stream().mapToInt(_.toInt).toArray
+        } else {
+          throw new java.lang.AssertionError()
+        }
+
+      case ValueCase.FLOAT_ARRAY =>
+        assert(valueType == classOf[Float])
+        val floatList = protoValue.getFloatArray.getElementList
+        val floatArray = new Array[Float](floatList.size())
+        for (i <- 0 until floatList.size()) {
+          floatArray(i) = floatList.get(i)
+        }
+        floatArray
+
+      case ValueCase.DOUBLE_ARRAY =>
+        val doubleList = protoValue.getDoubleArray.getElementList
+        if (valueType == classOf[Array[Double]]) {
+          doubleList.stream().mapToDouble(_.toDouble).toArray
+        } else if (valueType == classOf[Array[Float]]) {
+          val floatArray = new Array[Float](doubleList.size())
+          for (i <- 0 until doubleList.size()) {
+            floatArray(i) = doubleList.get(i).toFloat
+          }
+          floatArray
+        } else {
+          throw new java.lang.AssertionError()
+        }
+
+      case ValueCase.STR_ARRAY =>
+        assert(valueType == classOf[Array[String]])
+        protoValue.getStrArray.getElementList.toArray(Array[String]())
+
+      case ValueCase.BOOL_ARRAY =>
+        assert(valueType == classOf[Array[Boolean]])
+        val boolList = protoValue.getBoolArray.getElementList
+        val boolArray = new Array[Boolean](boolList.size())
+        for (i <- 0 until boolList.size()) {
+          boolArray(i) = boolList.get(i)
+        }
+        boolArray
+
+      case _ =>
+        throw InvalidPlanInput()
+    }
+    (param, extractedValue)
+  }
+
+  def setParams(instance: Params, protoParams: MlCommand.Params): Unit = {
+    protoParams.getParamsMap.forEach { (paramName, protoValue) =>
+      val (param, paramValue) = extractParamPair(
+        instance, paramName, protoValue
+      )
+      instance.set(param, paramValue)
+    }
+    protoParams.getDefaultParamsMap.forEach { (paramName, protoValue) =>
+      val (param, paramValue) = extractParamPair(
+        instance, paramName, protoValue
+      )
+      instance._setDefault(param, paramValue)
+    }
+  }
+
+  def handleRemoteCall(
+       sessionHolder: SessionHolder,
+       request: ExecutePlanRequest,
+       responseObserver: StreamObserver[ExecutePlanResponse]
+  ): Unit = {
+    val mlCommand = request.getPlan.getMlCommand
+    mlCommand.getOpTypeCase match {
+      case proto.MlCommand.OpTypeCase.CONSTRUCT_STAGE =>
+        val constructStage = mlCommand.getConstructStage
+        val stage = createStage(
+          constructStage.getUid,
+          constructStage.getClassName
+        )
+        val objectId = serverSideObjectManager.registerObject(stage)
+
+        val resp = ExecutePlanResponse.MlCommandResponse
+          .newBuilder.setServerSideObjectId(objectId).build()
+
+        buildMlResponse(request, responseObserver, Some(resp))
+
+      case proto.MlCommand.OpTypeCase.DESTRUCT_OBJECT =>
+        serverSideObjectManager.removeObject(mlCommand.getDestructObject.getId)
+
+      case proto.MlCommand.OpTypeCase.FIT =>
+        val estimator = serverSideObjectManager
+          .getObject(mlCommand.getFit.getId).asInstanceOf[Estimator[_]]
+        val inputDF = loadInputRelation(session, mlCommand.getFit.getInput)
+        val model = estimator.fit(inputDF)
+        val modelObjId = serverSideObjectManager.
+          registerObject(model.asInstanceOf[Object])
+
+        val resp = ExecutePlanResponse.MlCommandResponse
+          .newBuilder.setServerSideObjectId(modelObjId).build()
+
+        buildMlResponse(request, responseObserver, Some(resp))
+
+      case proto.MlCommand.OpTypeCase.TRANSFORM =>
+        val transformer = serverSideObjectManager.getObject(mlCommand.getTransform.getId)
+          .asInstanceOf[Transformer]
+        val inputDF = loadInputRelation(session, mlCommand.getTransform.getInput)
+        val transformedDF = transformer.transform(inputDF)
+        val transformedDFId = serverSideObjectManager.registerObject(transformedDF)
+
+        val resp = ExecutePlanResponse.MlCommandResponse
+          .newBuilder.setServerSideObjectId(transformedDFId).build()
+
+        buildMlResponse(request, responseObserver, Some(resp))
+
+      case proto.MlCommand.OpTypeCase.TRANSFER_PARAMS_TO_SERVER =>
+        val instance = serverSideObjectManager.getObject(mlCommand.getTransferParamsToServer.getId)
+          .asInstanceOf[Params]
+        val protoParams = mlCommand.getTransferParamsToServer.getParams
+        setParams(instance, protoParams)
+
+        buildMlResponse(request, responseObserver, None)
+
+      case _ =>
+        throw new UnsupportedOperationException(s"${mlCommand.getOpTypeCase} not supported.")
+    }
+  }
+  */
+
+  def parseArg(protoArg: proto.RemoteCall.ArgValue, sessionHolder: SessionHolder): Object = {
+    import scala.collection.JavaConverters._
+    import proto.RemoteCall.ArgValue
+    protoArg.getArgTypeCase match {
+      case ArgValue.ArgTypeCase.INT32_VALUE =>
+        lang.Integer.valueOf(protoArg.getInt32Value)
+      case ArgValue.ArgTypeCase.INT64_VALUE =>
+        lang.Long.valueOf(protoArg.getInt64Value)
+      case ArgValue.ArgTypeCase.FLOAT_VALUE =>
+        lang.Float.valueOf(protoArg.getFloatValue)
+      case ArgValue.ArgTypeCase.DOUBLE_VALUE =>
+        lang.Double.valueOf(protoArg.getDoubleValue)
+      case ArgValue.ArgTypeCase.STRING_VALUE =>
+        protoArg.getStringValue
+      case ArgValue.ArgTypeCase.BOOL_VALUE =>
+        lang.Boolean.valueOf(protoArg.getBoolValue)
+      case ArgValue.ArgTypeCase.RELATION =>
+        loadInputRelation(sessionHolder, protoArg.getRelation)
+      case ArgValue.ArgTypeCase.REMOTE_OBJECT =>
+        sessionHolder.serverSideObjectManager.getObject(protoArg.getRemoteObject.getId)
+      case ArgValue.ArgTypeCase.LIST =>
+        val protoValues = protoArg.getList.getElementList.asScala.toList
+        protoValues.map { x => parseArg(x, sessionHolder) }
+      case ArgValue.ArgTypeCase.MAP =>
+        val protoMap = protoArg.getMap.getMapMap().asScala.toMap
+        protoMap.mapValues(x => parseArg(x, sessionHolder))
+      case _ =>
+        throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Parse prototype message of argument list, returns a tuple of (values, types)
+   */
+  def parseArgs(
+      protoArgList: List[proto.RemoteCall.ArgValue],
+      sessionHolder: SessionHolder
+  ): Array[Object] = {
+    val argValues = new Array[Object](protoArgList.size)
+
+    for (i <- 0 until protoArgList.size) {
+      val protoValue = protoArgList(i)
+      argValues(i) = parseArg(protoValue, sessionHolder)
+    }
+    argValues
+  }
+
+  def serializeValue(value: Object, sessionHolder: SessionHolder): proto.RemoteCall.ArgValue = {
+    val protoBuilder = proto.RemoteCall.ArgValue.newBuilder()
+    value match {
+      case v: lang.Integer =>
+        protoBuilder.setInt32Value(v.intValue())
+      case v: lang.Long =>
+        protoBuilder.setInt64Value(v.longValue())
+      case v: lang.Float =>
+        protoBuilder.setFloatValue(v.floatValue())
+      case v: lang.Double =>
+        protoBuilder.setDoubleValue(v.doubleValue())
+      case v: String =>
+        protoBuilder.setStringValue(v)
+      case v: lang.Boolean =>
+        protoBuilder.setBoolValue(v)
+      case v: List[_] =>
+        // TODO
+        throw new UnsupportedOperationException()
+      case v: Map[_, _] =>
+        // TODO
+        throw new UnsupportedOperationException()
+      case v: Object =>
+        val instanceId = sessionHolder.serverSideObjectManager.registerObject(v)
+        protoBuilder.setRemoteObject(
+          RemoteCall.RemoteObject.newBuilder()
+            .setId(instanceId)
+            .build()
+        )
+    }
+    protoBuilder.build()
+  }
+
+  def invokeMethod(
+        instance: Object,
+        methodName: String,
+        argsProto: List[RemoteCall.ArgValue],
+        sessionHolder: SessionHolder): Option[RemoteCall.ArgValue] = {
+    val argValues = parseArgs(argsProto, sessionHolder)
+    val method = instance.getClass.getMethods().find { method =>

Review Comment:
   I wonder if this will eventually be a bottleneck for method invocation - maybe not. We could cache results, but that's complicating things



-- 
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-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org