mosche commented on code in PR #22157:
URL: https://github.com/apache/beam/pull/22157#discussion_r923107549


##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,33 +17,48 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import java.lang.reflect.Constructor;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag;
-import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
+  // default constructor to reflectively create static invoke expressions
+  private static final Constructor<StaticInvoke> STATIC_INVOKE_CONSTRUCTOR =
+      (Constructor<StaticInvoke>) StaticInvoke.class.getConstructors()[0];
+
+  static <T> ExpressionEncoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    return new ExpressionEncoder<>(serializer, deserializer, 
ClassTag.apply(clazz));
+  }
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    Expression serializer =
-        new EncoderHelpers.EncodeUsingBeamCoder<>(
-            new BoundReference(0, new ObjectType(clazz), true), coder);
-    Expression deserializer =
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(
-                new GetColumnByOrdinal(0, BinaryType), BinaryType, 
scala.Option.<String>empty()),
-            classTag,
-            coder);
-    return new ExpressionEncoder<>(serializer, deserializer, classTag);
+  /**
+   * Invoke method {@code fun} on Class {@code cls}, immediately propagating 
{@code null} if any
+   * input arg is {@code null}.
+   *
+   * <p>To address breaking interfaces between various version of Spark 3 
these are created

Review Comment:
   I tried doing that initially and decided it's doing more harm than good it 
as it required passing the spark session allround, including static encoder 
factories and stuff like that.  I'll keep as is.



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

Reply via email to