Hi Stefan,

There are some current limitations around how protobuf is embedded in Spark
Connect. One of the challenges there is that for compatibility reasons we
currently shade protobuf that then shades the
`prototobuf.GeneramtedMessage` class. The way to work around this is to
shade the protobuf library in your code following the same rules as in
Spark.

I have a fully working example here:
https://github.com/grundprinzip/spark-connect-appstate-example/tree/main

We definitely looking forward to improve the usability.

Hope this helps,
Martin

On Mon, Aug 28, 2023 at 4:19 PM Stefan Hagedorn <stefan.haged...@actian.com>
wrote:

> Hi everyone,
>
>
>
> Trying my luck here, after no success in the user mailing list :)
>
>
>
> I’m trying to use the "extension" feature of the Spark Connect
> CommandPlugin (Spark 3.4.1) [1].
>
>
>
> I created a simple protobuf message `MyMessage` that I want to send from
> the connect client-side to the connect server (where I registered my
> plugin).
>
>
>
> The source API for SparkSession class in `spark-connect-client-jvm`
> provides a method `execute` that accepts a `com.google.protobuf.Any` [2],
> so I packed the MyMessage object in an Any:
>
>
>
>     val spark = SparkSession.builder().remote("sc://localhost").build()
>
>
>
>   val cmd = com.test.MyMessage.newBuilder().setBlubb("hello world
> ").build()
>
>   val googleAny = com.google.protobuf.Any.pack(cmd)
>
>
>
>   spark.execute(googleAny)
>
>
>
>
>
> This compiles, but during execution I receive a NoSuchMethodError:
>
> java.lang.NoSuchMethodError: 'void
> org.apache.spark.sql.SparkSession.execute(com.google.protobuf.Any)'
>
>
>
> After looking around for a while after decompiling I found that
> spark-connect-client-jvm_2.12-3.4.1.jar!SparkSession#execute accepts a `
> org.sparkproject.connect.client.com.google.protobuf.Any` (instead of only
> the com.google.protobuf.Any).
>
>
>
> Am I missing something, how am I supposed to use this? Is there an
> additional build step or should I use a specific plugin? I'm using the
> sbt-protoc [3] plugin in my setup.
>
>
>
> Packing my message object `cmd` into an
> org.sparkproject.connect.client.com.google.protobuf.Any does not compile.
>
>
>
> Thanks,
>
> Stefan
>
>
>
>
>
> [1] https://github.com/apache/spark/pull/39291
>
> [2]
> https://github.com/apache/spark/blob/64c26b7cb9b4c770a3e056404e05f6b6603746ee/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala#L444
>
> [3] https://github.com/thesamet/sbt-protoc
>

Reply via email to