[ https://issues.apache.org/jira/browse/SPARK-37913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17795697#comment-17795697 ]
APeng Zhang edited comment on SPARK-37913 at 12/12/23 12:36 PM: ---------------------------------------------------------------- [~ally1221] I can reproduce this issue. h2. Solution: A simple approach to resolve this issue is to extract _UnaryTransformerExample.MyTransformer_ from {_}UnaryTransformerExample{_}, transforming it into a *top-level class* instead of a *nested class.* h2. Code to reproduce the issue: Adding these 2 blocks in _org.apache.spark.examples.ml.UnaryTransformerExample_ {code:java} val pipeline = new Pipeline().setStages(Array(myTransformer)) val pipelinePath = "~/tmp/ts/1"; pipeline.write.overwrite().save(pipelinePath) val pipeline2 = Pipeline.load(pipelinePath){code} {code:java} val model = pipeline.fit(data) val pipelineModelPath = "~/tmp/ts/2"; model.write.overwrite().save(pipelineModelPath) val model2 = PipelineModel.load(pipelineModelPath){code} there will be the exception: {code:java} 23/12/12 19:20:40 ERROR Instrumentation: java.lang.NoSuchMethodException: org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer.read() at java.base/java.lang.Class.newNoSuchMethodException(Class.java:660) at java.base/java.lang.Class.throwExceptionOrReturnNull(Class.java:1458) at java.base/java.lang.Class.getMethodHelper(Class.java:1544) at java.base/java.lang.Class.getMethod(Class.java:1450) at org.apache.spark.ml.util.DefaultParamsReader$.loadParamsInstanceReader(ReadWrite.scala:631) at org.apache.spark.ml.Pipeline$SharedReadWrite$.$anonfun$load$4(Pipeline.scala:276) at scala.collection.ArrayOps$.map$extension(ArrayOps.scala:934) at org.apache.spark.ml.Pipeline$SharedReadWrite$.$anonfun$load$3(Pipeline.scala:274){code} h2. Root cause: _Pipeline.PipelineWriter_ and _PipelineModel.PipelineModelWriter_ will persist the *class name* of each stage when saving _Pipeline_ or {_}PipelineModel{_}. In this example, the class name is {*}_org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer_{*}. The class name will be used by _PipelineReader_ and _PipelineModelReader_ to construct the Class object which represents the class, then invoke the _*read()*_ method. The problem here is for the *nested class* {_}UnaryTransformerExample.MyTransformer{_}, the Scala compiler will generate 2 different Java classes, the _*read()*_ method which is part of the Scala companion object will be a member of Java class {_}*org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer${color}*{_}, it is not a member of {_}*org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer*{_}. So there is the exception "{_}java.lang.NoSuchMethodException: org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer.read(){_}" when loading the object with _PipelineReader_ and {_}PipelineModelReader{_}. Note there is no such issue for a {*}non-nested class{*}, since the Scala compiler will generate only one Java class. h2. Java classes generated by Scala compiler: {code:java} public final class UnaryTransformerExample { ... public static class MyTransformer extends UnaryTransformer<Object, Object, MyTransformer> implements DefaultParamsWritable { ... public MLWriter write() { return DefaultParamsWritable.write$(this); } public void save(String path) throws IOException { MLWritable.save$((MLWritable)this, path); } ... } public static class MyTransformer$ implements DefaultParamsReadable<MyTransformer>, Serializable { public MLReader<UnaryTransformerExample.MyTransformer> read() { return DefaultParamsReadable.read$(this); } public Object load(String path) { return MLReadable.load$((MLReadable)this, path); } ... } } {code} was (Author: apeng): I can reproduce this issue. h2. Solution: A simple approach to resolve this issue is to extract _UnaryTransformerExample.MyTransformer_ from {_}UnaryTransformerExample{_}, transforming it into a *top-level class* instead of a *nested class.* h2. Code to reproduce the issue: Adding these 2 blocks in _org.apache.spark.examples.ml.UnaryTransformerExample_ {code:java} val pipeline = new Pipeline().setStages(Array(myTransformer)) val pipelinePath = "~/tmp/ts/1"; pipeline.write.overwrite().save(pipelinePath) val pipeline2 = Pipeline.load(pipelinePath){code} {code:java} val model = pipeline.fit(data) val pipelineModelPath = "~/tmp/ts/2"; model.write.overwrite().save(pipelineModelPath) val model2 = PipelineModel.load(pipelineModelPath){code} there will be the exception: {code:java} 23/12/12 19:20:40 ERROR Instrumentation: java.lang.NoSuchMethodException: org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer.read() at java.base/java.lang.Class.newNoSuchMethodException(Class.java:660) at java.base/java.lang.Class.throwExceptionOrReturnNull(Class.java:1458) at java.base/java.lang.Class.getMethodHelper(Class.java:1544) at java.base/java.lang.Class.getMethod(Class.java:1450) at org.apache.spark.ml.util.DefaultParamsReader$.loadParamsInstanceReader(ReadWrite.scala:631) at org.apache.spark.ml.Pipeline$SharedReadWrite$.$anonfun$load$4(Pipeline.scala:276) at scala.collection.ArrayOps$.map$extension(ArrayOps.scala:934) at org.apache.spark.ml.Pipeline$SharedReadWrite$.$anonfun$load$3(Pipeline.scala:274){code} h2. Root cause: _Pipeline.PipelineWriter_ and _PipelineModel.PipelineModelWriter_ will persist the *class name* of each stage when saving _Pipeline_ or {_}PipelineModel{_}. In this example, the class name is {*}_org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer_{*}. The class name will be used by _PipelineReader_ and _PipelineModelReader_ to construct the Class object which represents the class, then invoke the _*read()*_ method. The problem here is for the *nested class* {_}UnaryTransformerExample.MyTransformer{_}, the Scala compiler will generate 2 different Java classes, the _*read()*_ method which is part of the Scala companion object will be a member of Java class {_}*org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer{color:#FF0000}${color}*{_}, it is not a member of {_}*org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer*{_}. So there is the exception "{_}java.lang.NoSuchMethodException: org.apache.spark.examples.ml.UnaryTransformerExample$MyTransformer.read(){_}" when loading the object with _PipelineReader_ and {_}PipelineModelReader{_}. Note there is no such issue for a {*}non-nested class{*}, since the Scala compiler will generate only one Java class. h2. Java classes generated by Scala compiler: {code:java} public final class UnaryTransformerExample { ... public static class MyTransformer extends UnaryTransformer<Object, Object, MyTransformer> implements DefaultParamsWritable { ... public MLWriter write() { return DefaultParamsWritable.write$(this); } public void save(String path) throws IOException { MLWritable.save$((MLWritable)this, path); } ... } public static class MyTransformer$ implements DefaultParamsReadable<MyTransformer>, Serializable { public MLReader<UnaryTransformerExample.MyTransformer> read() { return DefaultParamsReadable.read$(this); } public Object load(String path) { return MLReadable.load$((MLReadable)this, path); } ... } } {code} > Null Pointer Exception when Loading ML Pipeline Model with Custom Transformer > ----------------------------------------------------------------------------- > > Key: SPARK-37913 > URL: https://issues.apache.org/jira/browse/SPARK-37913 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 3.1.2 > Environment: Spark 3.1.2, Scala 2.12, Java 11 > Reporter: Alana Young > Priority: Critical > Labels: MLPipelineModels, MLPipelines > > I am trying to create and persist a ML pipeline model using a custom Spark > transformer that I created based on the [Unary Transformer > example|https://github.com/apache/spark/blob/v3.1.2/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala] > provided by Spark. I am able to save and load the transformer. When I > include the custom transformer as a stage in a pipeline model, I can save the > model, but am unable to load it. Here is the stack trace of the exception: > > {code:java} > 01-14-2022 03:49:52 PM ERROR Instrumentation: java.lang.NullPointerException > at java.base/java.lang.reflect.Method.invoke(Method.java:559) at > org.apache.spark.ml.util.DefaultParamsReader$.loadParamsInstanceReader(ReadWrite.scala:631) > at > org.apache.spark.ml.Pipeline$SharedReadWrite$.$anonfun$load$4(Pipeline.scala:276) > at > scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238) at > scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) at > scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33) > at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198) at > scala.collection.TraversableLike.map(TraversableLike.scala:238) at > scala.collection.TraversableLike.map$(TraversableLike.scala:231) at > scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:198) at > org.apache.spark.ml.Pipeline$SharedReadWrite$.$anonfun$load$3(Pipeline.scala:274) > at > org.apache.spark.ml.util.Instrumentation$.$anonfun$instrumented$1(Instrumentation.scala:191) > at scala.util.Try$.apply(Try.scala:213) at > org.apache.spark.ml.util.Instrumentation$.instrumented(Instrumentation.scala:191) > at org.apache.spark.ml.Pipeline$SharedReadWrite$.load(Pipeline.scala:268) at > org.apache.spark.ml.PipelineModel$PipelineModelReader.$anonfun$load$7(Pipeline.scala:356) > at org.apache.spark.ml.MLEvents.withLoadInstanceEvent(events.scala:160) at > org.apache.spark.ml.MLEvents.withLoadInstanceEvent$(events.scala:155) at > org.apache.spark.ml.util.Instrumentation.withLoadInstanceEvent(Instrumentation.scala:42) > at > org.apache.spark.ml.PipelineModel$PipelineModelReader.$anonfun$load$6(Pipeline.scala:355) > at > org.apache.spark.ml.util.Instrumentation$.$anonfun$instrumented$1(Instrumentation.scala:191) > at scala.util.Try$.apply(Try.scala:213) at > org.apache.spark.ml.util.Instrumentation$.instrumented(Instrumentation.scala:191) > at > org.apache.spark.ml.PipelineModel$PipelineModelReader.load(Pipeline.scala:355) > at > org.apache.spark.ml.PipelineModel$PipelineModelReader.load(Pipeline.scala:349) > at org.apache.spark.ml.util.MLReadable.load(ReadWrite.scala:355) at > org.apache.spark.ml.util.MLReadable.load$(ReadWrite.scala:355) at > org.apache.spark.ml.PipelineModel$.load(Pipeline.scala:337) at > com.dtech.scala.pipeline.PipelineProcess.process(PipelineProcess.scala:122) > at com.dtech.scala.pipeline.PipelineProcess$.main(PipelineProcess.scala:448) > at com.dtech.scala.pipeline.PipelineProcess.main(PipelineProcess.scala) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.base/java.lang.reflect.Method.invoke(Method.java:566) at > org.apache.spark.deploy.worker.DriverWrapper$.main(DriverWrapper.scala:65) at > org.apache.spark.deploy.worker.DriverWrapper.main(DriverWrapper.scala){code} > > *Source Code* > [Unary > Transformer|https://gist.github.com/ally1221/ff10ec50f7ef98fb6cd365172195bfd5] > [Persist Unary Transformer & Pipeline > Model|https://gist.github.com/ally1221/42473cdc818a8cf795ac78d65d48ee14] -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org