[
https://issues.apache.org/jira/browse/SPARK-34452?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17489517#comment-17489517
]
zhengruifeng commented on SPARK-34452:
--------------------------------------
I can not reproduce this issue in 3.1.2, could you please provide a simple
example?
{code:java}
scala> import org.apache.spark.ml.classification._
import org.apache.spark.ml.classification._
scala> scala> val df =
spark.read.format("libsvm").load("/d0/Dev/Opensource/spark/data/mllib/sample_multiclass_classification_data.txt")
22/02/09 20:38:21 WARN LibSVMFileFormat: 'numFeatures' option not specified,
determining the number of features by going though the input. If you know the
number in advance, please specify it via 'numFeatures' option to avoid the
extra scan.
df: org.apache.spark.sql.DataFrame = [label: double, features: vector]
scala> scala> val classifier = new GBTClassifier().setMaxIter(2)
classifier: org.apache.spark.ml.classification.GBTClassifier = gbtc_e9ae5159908e
scala> val ovr = new OneVsRest().setClassifier(classifier)
ovr: org.apache.spark.ml.classification.OneVsRest = oneVsRest_466495ea9392
scala> val ovrm = ovr.fit(df)
ovrm: org.apache.spark.ml.classification.OneVsRestModel = OneVsRestModel:
uid=oneVsRest_466495ea9392, classifier=gbtc_e9ae5159908e, numClasses=3,
numFeatures=4
scala> ovrm.transform(df).show
22/02/09 20:38:27 WARN BLAS: Failed to load implementation from:
com.github.fommil.netlib.NativeSystemBLAS
22/02/09 20:38:27 WARN BLAS: Failed to load implementation from:
com.github.fommil.netlib.NativeRefBLAS
+-----+--------------------+--------------------+----------+
|label| features| rawPrediction|prediction|
+-----+--------------------+--------------------+----------+
| 1.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 1.0|
| 1.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 1.0|
| 1.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 1.0|
| 1.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 1.0|
| 0.0|(4,[0,1,2,3],[0.1...|[1.04768116880884...| 0.0|
| 1.0|(4,[0,2,3],[-0.83...|[-1.0476811688088...| 1.0|
| 2.0|(4,[0,1,2,3],[-1....|[-1.0476811688088...| 2.0|
| 2.0|(4,[0,1,2,3],[-1....|[-1.0476811688088...| 2.0|
| 1.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 1.0|
| 0.0|(4,[0,2,3],[0.611...|[1.04768116880884...| 0.0|
| 0.0|(4,[0,1,2,3],[0.2...|[1.04768116880884...| 0.0|
| 1.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 1.0|
| 1.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 1.0|
| 2.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 2.0|
| 2.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 2.0|
| 2.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 2.0|
| 1.0|(4,[0,2,3],[-0.94...|[-1.0476811688088...| 1.0|
| 2.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 2.0|
| 0.0|(4,[0,1,2,3],[0.1...|[1.04768116880884...| 0.0|
| 2.0|(4,[0,1,2,3],[-0....|[-1.0476811688088...| 2.0|
+-----+--------------------+--------------------+----------+
only showing top 20 rows{code}
> OneVsRest with GBTClassifier throws InternalCompilerException in 3.1.0
> ----------------------------------------------------------------------
>
> Key: SPARK-34452
> URL: https://issues.apache.org/jira/browse/SPARK-34452
> Project: Spark
> Issue Type: Bug
> Components: ML
> Affects Versions: 3.1.0
> Reporter: Danijel Zečević
> Priority: Major
>
> It looks like a problem with user defined function from OneVsRestModel.
> Log:
> {code}
> 2021-02-17 13:24:17.517 ERROR 711498 --- [818.0 (TID 818)]
> o.a.s.s.c.e.codegen.CodeGenerator : failed to compile:
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:361)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:234)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:446)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:313)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:235)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:204)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:82)
> ~[commons-compiler-3.1.2.jar:na]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1403)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1500)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1497)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$Segment.get(LocalCache.java:2257)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at org.sparkproject.guava.cache.LocalCache.get(LocalCache.java:4000)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1351)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:378)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:331)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:34)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1277)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:207)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:196)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.ScalaUDF.$anonfun$catalystConverter$3(ScalaUDF.scala:128)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.project_subExpr_0$(Unknown
> Source) ~[na:na]
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
> Source) ~[na:na]
> at
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:755)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer$$anon$1.hasNext(InMemoryRelation.scala:118)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
> ~[scala-library-2.12.10.jar:na]
> at
> org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:299)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1418)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1345)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1409)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1230)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:384)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:335)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.$anonfun$compute$1(SQLExecutionRDD.scala:52)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.internal.SQLConf$.withExistingConf(SQLConf.scala:122)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.compute(SQLExecutionRDD.scala:52)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.scheduler.Task.run(Task.scala:131)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> ~[na:na]
> at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> ~[na:na]
> at java.base/java.lang.Thread.run(Thread.java:834) ~[na:na]
> Caused by: org.codehaus.janino.InternalCompilerException: Two non-abstract
> methods "public int scala.collection.TraversableOnce.size()" have the same
> parameter types, declaring type and return type
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9112)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:8888)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8770)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8672)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:4737)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$8300(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4097)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4070)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:4902)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4070)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5253)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2559)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$2700(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1482)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.Java$LocalVariableDeclarationStatement.accept(Java.java:3351)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1532)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$1700(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1472)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.Java$Block.accept(Java.java:2756)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2455)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$1900(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1474)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3075)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1336)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1309)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:799)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:958)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$700(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:393)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1286)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1285)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:825)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:411)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:390)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1405)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:357)
> ~[janino-3.0.8.jar:na]
> ... 77 common frames omitted
> 2021-02-17 13:24:17.522 INFO 711498 --- [818.0 (TID 818)]
> o.a.s.s.c.e.codegen.CodeGenerator :
> /* 001 */ public java.lang.Object generate(Object[] references) {
> /* 002 */ return new SpecificUnsafeProjection(references);
> /* 003 */ }
> /* 004 */
> /* 005 */ class SpecificUnsafeProjection extends
> org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
> /* 006 */
> /* 007 */ private Object[] references;
> /* 008 */ private int value_ExternalMapToCatalyst_key_lambda_variable_1;
> /* 009 */ private double
> value_ExternalMapToCatalyst_value_lambda_variable_2;
> /* 010 */ private boolean globalIsNull_0;
> /* 011 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[]
> mutableStateArray_0 = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
> /* 012 */ private
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter[]
> mutableStateArray_1 = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter[2];
> /* 013 */
> /* 014 */ public SpecificUnsafeProjection(Object[] references) {
> /* 015 */ this.references = references;
> /* 016 */
> /* 017 */ mutableStateArray_0[0] = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
> /* 018 */ mutableStateArray_1[0] = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(mutableStateArray_0[0],
> 4);
> /* 019 */ mutableStateArray_1[1] = new
> org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(mutableStateArray_0[0],
> 8);
> /* 020 */
> /* 021 */ }
> /* 022 */
> /* 023 */ public void initialize(int partitionIndex) {
> /* 024 */
> /* 025 */ }
> /* 026 */
> /* 027 */ // Scala.Function1 need this
> /* 028 */ public java.lang.Object apply(java.lang.Object row) {
> /* 029 */ return apply((InternalRow) row);
> /* 030 */ }
> /* 031 */
> /* 032 */ public UnsafeRow apply(InternalRow i) {
> /* 033 */ mutableStateArray_0[0].reset();
> /* 034 */
> /* 035 */
> /* 036 */ mutableStateArray_0[0].zeroOutNullBytes();
> /* 037 */
> /* 038 */ MapData value_2 = ExternalMapToCatalyst_0(i);
> /* 039 */ if (globalIsNull_0) {
> /* 040 */ mutableStateArray_0[0].setNullAt(0);
> /* 041 */ } else {
> /* 042 */ final MapData tmpInput_0 = value_2;
> /* 043 */ if (tmpInput_0 instanceof UnsafeMapData) {
> /* 044 */ mutableStateArray_0[0].write(0, (UnsafeMapData) tmpInput_0);
> /* 045 */ } else {
> /* 046 */ // Remember the current cursor so that we can calculate how
> many bytes are
> /* 047 */ // written later.
> /* 048 */ final int previousCursor_0 =
> mutableStateArray_0[0].cursor();
> /* 049 */
> /* 050 */ // preserve 8 bytes to write the key array numBytes later.
> /* 051 */ mutableStateArray_0[0].grow(8);
> /* 052 */ mutableStateArray_0[0].increaseCursor(8);
> /* 053 */
> /* 054 */ // Remember the current cursor so that we can write
> numBytes of key array later.
> /* 055 */ final int tmpCursor_0 = mutableStateArray_0[0].cursor();
> /* 056 */
> /* 057 */
> /* 058 */ final ArrayData tmpInput_1 = tmpInput_0.keyArray();
> /* 059 */ if (tmpInput_1 instanceof UnsafeArrayData) {
> /* 060 */ mutableStateArray_0[0].write((UnsafeArrayData)
> tmpInput_1);
> /* 061 */ } else {
> /* 062 */ final int numElements_0 = tmpInput_1.numElements();
> /* 063 */ mutableStateArray_1[0].initialize(numElements_0);
> /* 064 */
> /* 065 */ for (int index_1 = 0; index_1 < numElements_0; index_1++)
> {
> /* 066 */ mutableStateArray_1[0].write(index_1,
> tmpInput_1.getInt(index_1));
> /* 067 */ }
> /* 068 */ }
> /* 069 */
> /* 070 */
> /* 071 */ // Write the numBytes of key array into the first 8 bytes.
> /* 072 */ Platform.putLong(
> /* 073 */ mutableStateArray_0[0].getBuffer(),
> /* 074 */ tmpCursor_0 - 8,
> /* 075 */ mutableStateArray_0[0].cursor() - tmpCursor_0);
> /* 076 */
> /* 077 */
> /* 078 */ final ArrayData tmpInput_2 = tmpInput_0.valueArray();
> /* 079 */ if (tmpInput_2 instanceof UnsafeArrayData) {
> /* 080 */ mutableStateArray_0[0].write((UnsafeArrayData)
> tmpInput_2);
> /* 081 */ } else {
> /* 082 */ final int numElements_1 = tmpInput_2.numElements();
> /* 083 */ mutableStateArray_1[1].initialize(numElements_1);
> /* 084 */
> /* 085 */ for (int index_2 = 0; index_2 < numElements_1; index_2++)
> {
> /* 086 */ mutableStateArray_1[1].write(index_2,
> tmpInput_2.getDouble(index_2));
> /* 087 */ }
> /* 088 */ }
> /* 089 */
> /* 090 */
> mutableStateArray_0[0].setOffsetAndSizeFromPreviousCursor(0,
> previousCursor_0);
> /* 091 */ }
> /* 092 */ }
> /* 093 */ return (mutableStateArray_0[0].getRow());
> /* 094 */ }
> /* 095 */
> /* 096 */
> /* 097 */ private MapData ExternalMapToCatalyst_0(InternalRow i) {
> /* 098 */ boolean isNull_1 = i.isNullAt(0);
> /* 099 */ scala.collection.immutable.Map value_1 = isNull_1 ?
> /* 100 */ null : ((scala.collection.immutable.Map)i.get(0, null));
> /* 101 */ MapData value_0 = null;
> /* 102 */ if (!isNull_1) {
> /* 103 */ final int length_0 = value_1.size();
> /* 104 */ final Object[] convertedKeys_0 = new Object[length_0];
> /* 105 */ final Object[] convertedValues_0 = new Object[length_0];
> /* 106 */ int index_0 = 0;
> /* 107 */ final scala.collection.Iterator entries_0 =
> value_1.iterator();
> /* 108 */ while(entries_0.hasNext()) {
> /* 109 */
> /* 110 */ final scala.Tuple2 entry_0 = (scala.Tuple2)
> entries_0.next();
> /* 111 */ value_ExternalMapToCatalyst_key_lambda_variable_1 =
> (Integer) entry_0._1();
> /* 112 */ value_ExternalMapToCatalyst_value_lambda_variable_2 =
> (Double) entry_0._2();
> /* 113 */
> /* 114 */
> /* 115 */
> /* 116 */
> /* 117 */
> /* 118 */ if (false) {
> /* 119 */ throw new RuntimeException("Cannot use null as map key!");
> /* 120 */ } else {
> /* 121 */ convertedKeys_0[index_0] = (Integer)
> value_ExternalMapToCatalyst_key_lambda_variable_1;
> /* 122 */ }
> /* 123 */
> /* 124 */
> /* 125 */ if (false) {
> /* 126 */ convertedValues_0[index_0] = null;
> /* 127 */ } else {
> /* 128 */ convertedValues_0[index_0] = (Double)
> value_ExternalMapToCatalyst_value_lambda_variable_2;
> /* 129 */ }
> /* 130 */
> /* 131 */ index_0++;
> /* 132 */ }
> /* 133 */
> /* 134 */ value_0 = new
> org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new
> org.apache.spark.sql.catalyst.util.GenericArrayData(convertedKeys_0), new
> org.apache.spark.sql.catalyst.util.GenericArrayData(convertedValues_0));
> /* 135 */ }
> /* 136 */ globalIsNull_0 = isNull_1;
> /* 137 */ return value_0;
> /* 138 */ }
> /* 139 */
> /* 140 */ }
> 2021-02-17 13:24:17.523 WARN 711498 --- [818.0 (TID 818)]
> org.apache.spark.storage.BlockManager : Putting block rdd_1866_0 failed
> due to exception org.apache.spark.SparkException: Failed to execute user
> defined function(OneVsRestModel$$Lambda$5255/0x0000000801a89440: () =>
> map<int,double>).
> 2021-02-17 13:24:17.524 WARN 711498 --- [818.0 (TID 818)]
> org.apache.spark.storage.BlockManager : Block rdd_1866_0 could not be
> removed as it was not found on disk or in memory
> 2021-02-17 13:24:17.527 ERROR 711498 --- [818.0 (TID 818)]
> org.apache.spark.executor.Executor : Exception in task 0.0 in stage
> 818.0 (TID 818)
> org.apache.spark.SparkException: Failed to execute user defined
> function(OneVsRestModel$$Lambda$5255/0x0000000801a89440: () =>
> map<int,double>)
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.project_subExpr_0$(Unknown
> Source) ~[na:na]
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
> Source) ~[na:na]
> at
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:755)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer$$anon$1.hasNext(InMemoryRelation.scala:118)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
> ~[scala-library-2.12.10.jar:na]
> at
> org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:299)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1418)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1345)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1409)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1230)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:384)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:335)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.$anonfun$compute$1(SQLExecutionRDD.scala:52)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.internal.SQLConf$.withExistingConf(SQLConf.scala:122)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.compute(SQLExecutionRDD.scala:52)
> ~[spark-sql_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.scheduler.Task.run(Task.scala:131)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
> ~[spark-core_2.12-3.1.0.jar:3.1.0]
> at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> ~[na:na]
> at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> ~[na:na]
> at java.base/java.lang.Thread.run(Thread.java:834) ~[na:na]
> Caused by: java.lang.RuntimeException: Error while encoding:
> org.codehaus.janino.InternalCompilerException: failed to compile:
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> externalmaptocatalyst(lambdavariable(ExternalMapToCatalyst_key, IntegerType,
> false, -1), lambdavariable(ExternalMapToCatalyst_key, IntegerType, false,
> -1), lambdavariable(ExternalMapToCatalyst_value, DoubleType, false, -2),
> lambdavariable(ExternalMapToCatalyst_value, DoubleType, false, -2), input[0,
> scala.collection.immutable.Map, true]) AS value#497
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:214)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:196)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.ScalaUDF.$anonfun$catalystConverter$3(ScalaUDF.scala:128)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> ... 53 common frames omitted
> Caused by: org.codehaus.janino.InternalCompilerException: failed to compile:
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1410)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1500)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1497)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$Segment.get(LocalCache.java:2257)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at org.sparkproject.guava.cache.LocalCache.get(LocalCache.java:4000)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.sparkproject.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
> ~[spark-network-common_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1351)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:378)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:331)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:34)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1277)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:207)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> ... 55 common frames omitted
> Caused by: org.codehaus.janino.InternalCompilerException: Compiling
> "GeneratedClass": Two non-abstract methods "public int
> scala.collection.TraversableOnce.size()" have the same parameter types,
> declaring type and return type
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:361)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:234)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:446)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:313)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:235)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:204)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:82)
> ~[commons-compiler-3.1.2.jar:na]
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1403)
> ~[spark-catalyst_2.12-3.1.0.jar:3.1.0]
> ... 70 common frames omitted
> Caused by: org.codehaus.janino.InternalCompilerException: Two non-abstract
> methods "public int scala.collection.TraversableOnce.size()" have the same
> parameter types, declaring type and return type
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9112)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:8888)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8770)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8672)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:4737)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$8300(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4097)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4070)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:4902)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4070)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5253)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2559)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$2700(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1482)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.Java$LocalVariableDeclarationStatement.accept(Java.java:3351)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1532)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$1700(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1472)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.Java$Block.accept(Java.java:2756)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2455)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$1900(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1474)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3075)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1336)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1309)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:799)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:958)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$700(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:393)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1286)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1285)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:825)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:411)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:212)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:390)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at
> org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1405)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> ~[janino-3.0.8.jar:na]
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:357)
> ~[janino-3.0.8.jar:na]
> ... 77 common frames omitted
> 2021-02-17 13:24:17.540 WARN 711498 --- [result-getter-2]
> o.apache.spark.scheduler.TaskSetManager : Lost task 0.0 in stage 818.0 (TID
> 818) (10.99.10.158 executor driver): org.apache.spark.SparkException: Failed
> to execute user defined
> function(OneVsRestModel$$Lambda$5255/0x0000000801a89440: () =>
> map<int,double>)
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.project_subExpr_0$(Unknown
> Source)
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
> Source)
> at
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> at
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:755)
> at
> org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer$$anon$1.hasNext(InMemoryRelation.scala:118)
> at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
> at
> org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
> at
> org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:299)
> at
> org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1418)
> at
> org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1345)
> at
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1409)
> at
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1230)
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:384)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:335)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.$anonfun$compute$1(SQLExecutionRDD.scala:52)
> at
> org.apache.spark.sql.internal.SQLConf$.withExistingConf(SQLConf.scala:122)
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.compute(SQLExecutionRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
> at org.apache.spark.scheduler.Task.run(Task.scala:131)
> at
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> at java.base/java.lang.Thread.run(Thread.java:834)
> Caused by: java.lang.RuntimeException: Error while encoding:
> org.codehaus.janino.InternalCompilerException: failed to compile:
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> externalmaptocatalyst(lambdavariable(ExternalMapToCatalyst_key, IntegerType,
> false, -1), lambdavariable(ExternalMapToCatalyst_key, IntegerType, false,
> -1), lambdavariable(ExternalMapToCatalyst_value, DoubleType, false, -2),
> lambdavariable(ExternalMapToCatalyst_value, DoubleType, false, -2), input[0,
> scala.collection.immutable.Map, true]) AS value#497
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:214)
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:196)
> at
> org.apache.spark.sql.catalyst.expressions.ScalaUDF.$anonfun$catalystConverter$3(ScalaUDF.scala:128)
> ... 53 more
> Caused by: org.codehaus.janino.InternalCompilerException: failed to compile:
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1410)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1500)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1497)
> at
> org.sparkproject.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
> at
> org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
> at
> org.sparkproject.guava.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
> at
> org.sparkproject.guava.cache.LocalCache$Segment.get(LocalCache.java:2257)
> at org.sparkproject.guava.cache.LocalCache.get(LocalCache.java:4000)
> at
> org.sparkproject.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004)
> at
> org.sparkproject.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1351)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:378)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:331)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:34)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1277)
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:207)
> ... 55 more
> Caused by: org.codehaus.janino.InternalCompilerException: Compiling
> "GeneratedClass": Two non-abstract methods "public int
> scala.collection.TraversableOnce.size()" have the same parameter types,
> declaring type and return type
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:361)
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:234)
> at
> org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:446)
> at
> org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:313)
> at
> org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:235)
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:204)
> at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:82)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1403)
> ... 70 more
> Caused by: org.codehaus.janino.InternalCompilerException: Two non-abstract
> methods "public int scala.collection.TraversableOnce.size()" have the same
> parameter types, declaring type and return type
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9112)
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:8888)
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8770)
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8672)
> at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:4737)
> at org.codehaus.janino.UnitCompiler.access$8300(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4097)
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4070)
> at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:4902)
> at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4070)
> at
> org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5253)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2559)
> at org.codehaus.janino.UnitCompiler.access$2700(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1482)
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1466)
> at
> org.codehaus.janino.Java$LocalVariableDeclarationStatement.accept(Java.java:3351)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1532)
> at org.codehaus.janino.UnitCompiler.access$1700(UnitCompiler.java:212)
> at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1472)
> at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1466)
> at org.codehaus.janino.Java$Block.accept(Java.java:2756)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2455)
> at org.codehaus.janino.UnitCompiler.access$1900(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1474)
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1466)
> at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3075)
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1336)
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1309)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:799)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:958)
> at org.codehaus.janino.UnitCompiler.access$700(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:393)
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:385)
> at
> org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1286)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1285)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:825)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:411)
> at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:390)
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:385)
> at
> org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1405)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:357)
> ... 77 more
> 2021-02-17 13:24:17.541 ERROR 711498 --- [result-getter-2]
> o.apache.spark.scheduler.TaskSetManager : Task 0 in stage 818.0 failed 1
> times; aborting job
> 2021-02-17 13:24:17.541 INFO 711498 --- [result-getter-2]
> o.a.spark.scheduler.TaskSchedulerImpl : Removed TaskSet 818.0, whose tasks
> have all completed, from pool
> 2021-02-17 13:24:17.542 INFO 711498 --- [uler-event-loop]
> o.a.spark.scheduler.TaskSchedulerImpl : Cancelling stage 818
> 2021-02-17 13:24:17.542 INFO 711498 --- [uler-event-loop]
> o.a.spark.scheduler.TaskSchedulerImpl : Killing all running tasks in stage
> 818: Stage cancelled
> 2021-02-17 13:24:17.543 INFO 711498 --- [uler-event-loop]
> org.apache.spark.scheduler.DAGScheduler : ShuffleMapStage 818 (map at
> MulticlassMetrics.scala:52) failed in 0.160 s due to Job aborted due to stage
> failure: Task 0 in stage 818.0 failed 1 times, most recent failure: Lost task
> 0.0 in stage 818.0 (TID 818) (10.99.10.158 executor driver):
> org.apache.spark.SparkException: Failed to execute user defined
> function(OneVsRestModel$$Lambda$5255/0x0000000801a89440: () =>
> map<int,double>)
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.project_subExpr_0$(Unknown
> Source)
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
> Source)
> at
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> at
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:755)
> at
> org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer$$anon$1.hasNext(InMemoryRelation.scala:118)
> at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
> at
> org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
> at
> org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:299)
> at
> org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1418)
> at
> org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1345)
> at
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1409)
> at
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1230)
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:384)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:335)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.$anonfun$compute$1(SQLExecutionRDD.scala:52)
> at
> org.apache.spark.sql.internal.SQLConf$.withExistingConf(SQLConf.scala:122)
> at
> org.apache.spark.sql.execution.SQLExecutionRDD.compute(SQLExecutionRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
> at
> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
> at org.apache.spark.scheduler.Task.run(Task.scala:131)
> at
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> at java.base/java.lang.Thread.run(Thread.java:834)
> Caused by: java.lang.RuntimeException: Error while encoding:
> org.codehaus.janino.InternalCompilerException: failed to compile:
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> externalmaptocatalyst(lambdavariable(ExternalMapToCatalyst_key, IntegerType,
> false, -1), lambdavariable(ExternalMapToCatalyst_key, IntegerType, false,
> -1), lambdavariable(ExternalMapToCatalyst_value, DoubleType, false, -2),
> lambdavariable(ExternalMapToCatalyst_value, DoubleType, false, -2), input[0,
> scala.collection.immutable.Map, true]) AS value#497
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:214)
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:196)
> at
> org.apache.spark.sql.catalyst.expressions.ScalaUDF.$anonfun$catalystConverter$3(ScalaUDF.scala:128)
> ... 53 more
> Caused by: org.codehaus.janino.InternalCompilerException: failed to compile:
> org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass":
> Two non-abstract methods "public int scala.collection.TraversableOnce.size()"
> have the same parameter types, declaring type and return type
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1410)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1500)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1497)
> at
> org.sparkproject.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
> at
> org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
> at
> org.sparkproject.guava.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
> at
> org.sparkproject.guava.cache.LocalCache$Segment.get(LocalCache.java:2257)
> at org.sparkproject.guava.cache.LocalCache.get(LocalCache.java:4000)
> at
> org.sparkproject.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004)
> at
> org.sparkproject.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1351)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:378)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:331)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:34)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1277)
> at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer.apply(ExpressionEncoder.scala:207)
> ... 55 more
> Caused by: org.codehaus.janino.InternalCompilerException: Compiling
> "GeneratedClass": Two non-abstract methods "public int
> scala.collection.TraversableOnce.size()" have the same parameter types,
> declaring type and return type
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:361)
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:234)
> at
> org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:446)
> at
> org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:313)
> at
> org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:235)
> at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:204)
> at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:82)
> at
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1403)
> ... 70 more
> Caused by: org.codehaus.janino.InternalCompilerException: Two non-abstract
> methods "public int scala.collection.TraversableOnce.size()" have the same
> parameter types, declaring type and return type
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9112)
> at
> org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:8888)
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8770)
> at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:8672)
> at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:4737)
> at org.codehaus.janino.UnitCompiler.access$8300(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4097)
> at
> org.codehaus.janino.UnitCompiler$12.visitMethodInvocation(UnitCompiler.java:4070)
> at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:4902)
> at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4070)
> at
> org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5253)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2559)
> at org.codehaus.janino.UnitCompiler.access$2700(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1482)
> at
> org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1466)
> at
> org.codehaus.janino.Java$LocalVariableDeclarationStatement.accept(Java.java:3351)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1532)
> at org.codehaus.janino.UnitCompiler.access$1700(UnitCompiler.java:212)
> at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1472)
> at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1466)
> at org.codehaus.janino.Java$Block.accept(Java.java:2756)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2455)
> at org.codehaus.janino.UnitCompiler.access$1900(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1474)
> at
> org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1466)
> at org.codehaus.janino.Java$IfStatement.accept(Java.java:2926)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1466)
> at
> org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1546)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3075)
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1336)
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1309)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:799)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:958)
> at org.codehaus.janino.UnitCompiler.access$700(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:393)
> at
> org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:385)
> at
> org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1286)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> at
> org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1285)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:825)
> at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:411)
> at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:212)
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:390)
> at
> org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:385)
> at
> org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1405)
> at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:385)
> at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:357)
> ... 77 more
> {code}
--
This message was sent by Atlassian Jira
(v8.20.1#820001)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]