Skip to content

Commit bc7bfbc

Browse files
hvanhovellHyukjinKwon
authored andcommitted
[SPARK-49274][CONNECT] Support java serialization based encoders
### What changes were proposed in this pull request? This PR adds Encoders.javaSerialization to connect. It does this by creating an encoder that does not really encode data, but that transforms into something that can be encoded. This is also useful for situations in which we want to define custom serialization logic. This PR is also a stepping stone for adding Kryo based serialization. ### Why are the changes needed? This change increases parity between the connect and classic scala interfaces. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests for both Arrow and Expression encoders. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47806 from hvanhovell/SPARK-49274. Authored-by: Herman van Hovell <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
1 parent af4c738 commit bc7bfbc

File tree

9 files changed

+165
-14
lines changed

9 files changed

+165
-14
lines changed

connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala

Lines changed: 28 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -16,10 +16,11 @@
1616
*/
1717
package org.apache.spark.sql
1818

19+
import scala.reflect.ClassTag
1920
import scala.reflect.runtime.universe.TypeTag
2021

2122
import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection}
22-
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder => RowEncoderFactory}
23+
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, JavaSerializationCodec, RowEncoder => RowEncoderFactory}
2324
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._
2425
import org.apache.spark.sql.types.StructType
2526

@@ -176,6 +177,32 @@ object Encoders {
176177
*/
177178
def row(schema: StructType): Encoder[Row] = RowEncoderFactory.encoderFor(schema)
178179

180+
/**
181+
* (Scala-specific) Creates an encoder that serializes objects of type T using generic Java
182+
* serialization. This encoder maps T into a single byte array (binary) field.
183+
*
184+
* T must be publicly accessible.
185+
*
186+
* @note
187+
* This is extremely inefficient and should only be used as the last resort.
188+
* @since 4.0.0
189+
*/
190+
def javaSerialization[T: ClassTag]: Encoder[T] = {
191+
TransformingEncoder(implicitly[ClassTag[T]], BinaryEncoder, JavaSerializationCodec)
192+
}
193+
194+
/**
195+
* Creates an encoder that serializes objects of type T using generic Java serialization. This
196+
* encoder maps T into a single byte array (binary) field.
197+
*
198+
* T must be publicly accessible.
199+
*
200+
* @note
201+
* This is extremely inefficient and should only be used as the last resort.
202+
* @since 4.0.0
203+
*/
204+
def javaSerialization[T](clazz: Class[T]): Encoder[T] = javaSerialization(ClassTag[T](clazz))
205+
179206
private def tupleEncoder[T](encoders: Encoder[_]*): Encoder[T] = {
180207
ProductEncoder.tuple(encoders.asInstanceOf[Seq[AgnosticEncoder[_]]]).asInstanceOf[Encoder[T]]
181208
}

connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala

Lines changed: 28 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -30,11 +30,11 @@ import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
3030
import org.apache.arrow.vector.VarBinaryVector
3131
import org.scalatest.BeforeAndAfterAll
3232

33-
import org.apache.spark.SparkUnsupportedOperationException
34-
import org.apache.spark.sql.{AnalysisException, Row}
33+
import org.apache.spark.{sql, SparkUnsupportedOperationException}
34+
import org.apache.spark.sql.{AnalysisException, Encoders, Row}
3535
import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, JavaTypeInference, ScalaReflection}
36-
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, OuterScopes}
37-
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, NullEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, RowEncoder, ScalaDecimalEncoder, StringEncoder, TimestampEncoder, UDTEncoder, YearMonthIntervalEncoder}
36+
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, Codec, OuterScopes}
37+
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, NullEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, RowEncoder, ScalaDecimalEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, YearMonthIntervalEncoder}
3838
import org.apache.spark.sql.catalyst.encoders.RowEncoder.{encoderFor => toRowEncoder}
3939
import org.apache.spark.sql.catalyst.util.{DateFormatter, SparkStringUtils, TimestampFormatter}
4040
import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_SECOND
@@ -44,7 +44,7 @@ import org.apache.spark.sql.catalyst.util.SparkIntervalUtils._
4444
import org.apache.spark.sql.connect.client.CloseableIterator
4545
import org.apache.spark.sql.connect.client.arrow.FooEnum.FooEnum
4646
import org.apache.spark.sql.test.ConnectFunSuite
47-
import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, IntegerType, Metadata, SQLUserDefinedType, StructType, UserDefinedType, YearMonthIntervalType}
47+
import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, IntegerType, Metadata, SQLUserDefinedType, StringType, StructType, UserDefinedType, YearMonthIntervalType}
4848

4949
/**
5050
* Tests for encoding external data to and from arrow.
@@ -769,6 +769,24 @@ class ArrowEncoderSuite extends ConnectFunSuite with BeforeAndAfterAll {
769769
}
770770
}
771771

772+
test("java serialization") {
773+
val encoder = sql.encoderFor(Encoders.javaSerialization[(Int, String)])
774+
roundTripAndCheckIdentical(encoder) { () =>
775+
Iterator.tabulate(10)(i => (i, "itr_" + i))
776+
}
777+
}
778+
779+
test("transforming encoder") {
780+
val schema = new StructType()
781+
.add("key", IntegerType)
782+
.add("value", StringType)
783+
val encoder =
784+
TransformingEncoder(classTag[(Int, String)], toRowEncoder(schema), () => new TestCodec)
785+
roundTripAndCheckIdentical(encoder) { () =>
786+
Iterator.tabulate(10)(i => (i, "v" + i))
787+
}
788+
}
789+
772790
/* ******************************************************************** *
773791
* Arrow deserialization upcasting
774792
* ******************************************************************** */
@@ -1136,3 +1154,8 @@ class UDTNotSupported extends UserDefinedType[UDTNotSupportedClass] {
11361154
case i: Int => UDTNotSupportedClass(i)
11371155
}
11381156
}
1157+
1158+
class TestCodec extends Codec[(Int, String), Row] {
1159+
override def encode(in: (Int, String)): Row = Row(in._1, in._2)
1160+
override def decode(out: Row): (Int, String) = (out.getInt(0), out.getString(1))
1161+
}

sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -247,5 +247,20 @@ object AgnosticEncoders {
247247
ScalaDecimalEncoder(DecimalType.SYSTEM_DEFAULT)
248248
val DEFAULT_JAVA_DECIMAL_ENCODER: JavaDecimalEncoder =
249249
JavaDecimalEncoder(DecimalType.SYSTEM_DEFAULT, lenientSerialization = false)
250-
}
251250

251+
/**
252+
* Encoder that transforms external data into a representation that can be further processed by
253+
* another encoder. This is fallback for scenarios where objects can't be represented using
254+
* standard encoders, an example of this is where we use a different (opaque) serialization
255+
* format (i.e. java serialization, kryo serialization, or protobuf).
256+
*/
257+
case class TransformingEncoder[I, O](
258+
clsTag: ClassTag[I],
259+
transformed: AgnosticEncoder[O],
260+
codecProvider: () => Codec[_ >: I, O]) extends AgnosticEncoder[I] {
261+
override def isPrimitive: Boolean = transformed.isPrimitive
262+
override def dataType: DataType = transformed.dataType
263+
override def schema: StructType = transformed.schema
264+
override def isStruct: Boolean = transformed.isStruct
265+
}
266+
}
Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,42 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.spark.sql.catalyst.encoders
18+
19+
import org.apache.spark.util.SparkSerDeUtils
20+
21+
/**
22+
* Codec for doing conversions between two representations.
23+
*
24+
* @tparam I input type (typically the external representation of the data.
25+
* @tparam O output type (typically the internal representation of the data.
26+
*/
27+
trait Codec[I, O] {
28+
def encode(in: I): O
29+
def decode(out: O): I
30+
}
31+
32+
/**
33+
* A codec that uses Java Serialization as its output format.
34+
*/
35+
class JavaSerializationCodec[I] extends Codec[I, Array[Byte]] {
36+
override def encode(in: I): Array[Byte] = SparkSerDeUtils.serialize(in)
37+
override def decode(out: Array[Byte]): I = SparkSerDeUtils.deserialize(out)
38+
}
39+
40+
object JavaSerializationCodec extends (() => Codec[Any, Array[Byte]]) {
41+
override def apply(): Codec[Any, Array[Byte]] = new JavaSerializationCodec[Any]
42+
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,10 +19,10 @@ package org.apache.spark.sql.catalyst
1919

2020
import org.apache.spark.sql.catalyst.{expressions => exprs}
2121
import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedExtractValue}
22-
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders}
23-
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedLeafEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, UDTEncoder, YearMonthIntervalEncoder}
22+
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders, Codec}
23+
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedLeafEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, YearMonthIntervalEncoder}
2424
import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{externalDataTypeFor, isNativeEncoder}
25-
import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, IsNull, MapKeys, MapValues, UpCast}
25+
import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, IsNull, Literal, MapKeys, MapValues, UpCast}
2626
import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, InitializeJavaBean, Invoke, NewInstance, StaticInvoke, UnresolvedCatalystToExternalMap, UnresolvedMapObjects, WrapOption}
2727
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, IntervalUtils}
2828
import org.apache.spark.sql.types._
@@ -410,6 +410,13 @@ object DeserializerBuildHelper {
410410
val newInstance = NewInstance(cls, Nil, ObjectType(cls), propagateNull = false)
411411
val result = InitializeJavaBean(newInstance, setters.toMap)
412412
exprs.If(IsNull(path), exprs.Literal.create(null, ObjectType(cls)), result)
413+
414+
case TransformingEncoder(tag, encoder, provider) =>
415+
Invoke(
416+
Literal.create(provider(), ObjectType(classOf[Codec[_, _]])),
417+
"decode",
418+
ObjectType(tag.runtimeClass),
419+
createDeserializer(encoder, path, walkedTypePath) :: Nil)
413420
}
414421

415422
private def deserializeArray(

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,10 @@ import scala.language.existentials
2121

2222
import org.apache.spark.sql.catalyst.{expressions => exprs}
2323
import org.apache.spark.sql.catalyst.DeserializerBuildHelper.expressionWithNullSafety
24-
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders}
25-
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLeafEncoder, BoxedLongEncoder, BoxedShortEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveLeafEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, UDTEncoder, YearMonthIntervalEncoder}
24+
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders, Codec}
25+
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLeafEncoder, BoxedLongEncoder, BoxedShortEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveLeafEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, YearMonthIntervalEncoder}
2626
import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{externalDataTypeFor, isNativeEncoder, lenientExternalDataTypeFor}
27-
import org.apache.spark.sql.catalyst.expressions.{BoundReference, CheckOverflow, CreateNamedStruct, Expression, IsNull, KnownNotNull, UnsafeArrayData}
27+
import org.apache.spark.sql.catalyst.expressions.{BoundReference, CheckOverflow, CreateNamedStruct, Expression, IsNull, KnownNotNull, Literal, UnsafeArrayData}
2828
import org.apache.spark.sql.catalyst.expressions.objects._
2929
import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, GenericArrayData, IntervalUtils}
3030
import org.apache.spark.sql.internal.SQLConf
@@ -397,6 +397,14 @@ object SerializerBuildHelper {
397397
f.name -> createSerializer(f.enc, fieldValue)
398398
}
399399
createSerializerForObject(input, serializedFields)
400+
401+
case TransformingEncoder(_, encoder, codecProvider) =>
402+
val encoded = Invoke(
403+
Literal(codecProvider(), ObjectType(classOf[Codec[_, _]])),
404+
"encode",
405+
externalDataTypeFor(encoder),
406+
input :: Nil)
407+
createSerializer(encoder, encoded)
400408
}
401409

402410
private def serializerForArray(

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,13 +22,15 @@ import java.sql.{Date, Timestamp}
2222
import java.util.Arrays
2323

2424
import scala.collection.mutable.ArrayBuffer
25+
import scala.reflect.classTag
2526
import scala.reflect.runtime.universe.TypeTag
2627

2728
import org.apache.spark.{SPARK_DOC_ROOT, SparkArithmeticException, SparkRuntimeException, SparkUnsupportedOperationException}
2829
import org.apache.spark.sql.{Encoder, Encoders, Row}
2930
import org.apache.spark.sql.catalyst.{FooClassWithEnum, FooEnum, OptionalData, PrimitiveData, ScroogeLikeExample}
3031
import org.apache.spark.sql.catalyst.analysis.AnalysisTest
3132
import org.apache.spark.sql.catalyst.dsl.plans._
33+
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, TransformingEncoder}
3234
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, NaNvl}
3335
import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest
3436
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
@@ -550,6 +552,18 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes
550552
encodeDecodeTest(FooClassWithEnum(1, FooEnum.E1), "case class with Int and scala Enum")
551553
encodeDecodeTest(FooEnum.E1, "scala Enum")
552554

555+
test("transforming encoder") {
556+
val encoder = ExpressionEncoder(TransformingEncoder(
557+
classTag[(Long, Long)],
558+
BinaryEncoder,
559+
JavaSerializationCodec))
560+
.resolveAndBind()
561+
assert(encoder.schema == new StructType().add("value", BinaryType))
562+
val toRow = encoder.createSerializer()
563+
val fromRow = encoder.createDeserializer()
564+
assert(fromRow(toRow((11, 14))) == (11, 14))
565+
}
566+
553567
// Scala / Java big decimals ----------------------------------------------------------
554568

555569
encodeDecodeTest(BigDecimal(("9" * 20) + "." + "9" * 18),

sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -359,6 +359,13 @@ object ArrowDeserializers {
359359
}
360360
}
361361

362+
case (TransformingEncoder(_, encoder, provider), v) =>
363+
new Deserializer[Any] {
364+
private[this] val codec = provider()
365+
private[this] val deserializer = deserializerFor(encoder, v, timeZoneId)
366+
override def get(i: Int): Any = codec.decode(deserializer.get(i))
367+
}
368+
362369
case (CalendarIntervalEncoder | VariantEncoder | _: UDTEncoder[_], _) =>
363370
throw ExecutionErrors.unsupportedDataTypeError(encoder.dataType)
364371

sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ import org.apache.arrow.vector.util.Text
3535

3636
import org.apache.spark.sql.Row
3737
import org.apache.spark.sql.catalyst.DefinedByConstructorParams
38-
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder
38+
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, Codec}
3939
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._
4040
import org.apache.spark.sql.catalyst.util.{SparkDateTimeUtils, SparkIntervalUtils}
4141
import org.apache.spark.sql.connect.client.CloseableIterator
@@ -442,6 +442,14 @@ object ArrowSerializer {
442442
o => getter.invoke(o)
443443
}
444444

445+
case (TransformingEncoder(_, encoder, provider), v) =>
446+
new Serializer {
447+
private[this] val codec = provider().asInstanceOf[Codec[Any, Any]]
448+
private[this] val delegate: Serializer = serializerFor(encoder, v)
449+
override def write(index: Int, value: Any): Unit =
450+
delegate.write(index, codec.encode(value))
451+
}
452+
445453
case (CalendarIntervalEncoder | VariantEncoder | _: UDTEncoder[_], _) =>
446454
throw ExecutionErrors.unsupportedDataTypeError(encoder.dataType)
447455

0 commit comments

Comments
 (0)