Emil Ejbyfeldt created SPARK-45820:
--------------------------------------

             Summary: Support encoding of scala.collection.immutable.ArraySeq
                 Key: SPARK-45820
                 URL: https://issues.apache.org/jira/browse/SPARK-45820
             Project: Spark
          Issue Type: Bug
          Components: SQL
    Affects Versions: 3.5.0, 3.4.1, 3.4.0, 4.0.0
            Reporter: Emil Ejbyfeldt


Trying to use `scala.collection.immutable.ArraySeq` will currently derive the 
encoder as it a subtype of scala.collection.Seq but will then fail at runtime 
since the builder interface is different then for other Seq.


{code:java}
scala> 
spark.createDataset(Seq(scala.collection.immutable.ArraySeq(1,2,3))).collect()
23/11/07 09:44:39 INFO SharedState: Setting hive.metastore.warehouse.dir 
('null') to the value of spark.sql.warehouse.dir.
23/11/07 09:44:39 INFO SharedState: Warehouse path is 
'file:/home/eejbyfeldt/spark-warehouse'.
23/11/07 09:44:40 INFO CodeGenerator: Code generated in 188.491705 ms
23/11/07 09:44:40 INFO CodeGenerator: Code generated in 14.382264 ms
23/11/07 09:44:40 ERROR CodeGenerator: failed to compile: 
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, 
Column 101: No applicable constructor/method found for zero actual parameters; 
candidates are: "public scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(scala.reflect.ClassTag)", 
"public scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(java.lang.Object)", "public 
abstract scala.collection.mutable.Builder 
scala.collection.EvidenceIterableFactory.newBuilder(java.lang.Object)"
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, 
Column 101: No applicable constructor/method found for zero actual parameters; 
candidates are: "public scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(scala.reflect.ClassTag)", 
"public scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(java.lang.Object)", "public 
abstract scala.collection.mutable.Builder 
scala.collection.EvidenceIterableFactory.newBuilder(java.lang.Object)"
    at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:13014)
    at 
org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9615)
    at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9475)
    at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9391)
    at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5232)
    at org.codehaus.janino.UnitCompiler.access$9300(UnitCompiler.java:236)
    at 
org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4735)
    at 
org.codehaus.janino.UnitCompiler$16.visitMethodInvocation(UnitCompiler.java:4711)
    at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:5470)
    at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:4711)
    at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:5854)
    at org.codehaus.janino.UnitCompiler.access$3800(UnitCompiler.java:236)
    at org.codehaus.janino.UnitCompiler$7.visitRvalue(UnitCompiler.java:2766)
    at org.codehaus.janino.UnitCompiler$7.visitRvalue(UnitCompiler.java:2754)
    at org.codehaus.janino.Java$Rvalue.accept(Java.java:4498)
    at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2754)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2741)
    at org.codehaus.janino.UnitCompiler.access$2700(UnitCompiler.java:236)
    at 
org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1589)
    at 
org.codehaus.janino.UnitCompiler$6.visitLocalVariableDeclarationStatement(UnitCompiler.java:1575)
    at 
org.codehaus.janino.Java$LocalVariableDeclarationStatement.accept(Java.java:3842)
    at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1575)
    at 
org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1661)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1646)
    at org.codehaus.janino.UnitCompiler.access$1700(UnitCompiler.java:236)
    at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1579)
    at org.codehaus.janino.UnitCompiler$6.visitBlock(UnitCompiler.java:1575)
    at org.codehaus.janino.Java$Block.accept(Java.java:3115)
    at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1575)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2649)
    at org.codehaus.janino.UnitCompiler.access$1900(UnitCompiler.java:236)
    at 
org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1581)
    at 
org.codehaus.janino.UnitCompiler$6.visitIfStatement(UnitCompiler.java:1575)
    at org.codehaus.janino.Java$IfStatement.accept(Java.java:3284)
    at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:1575)
    at 
org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1661)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3658)
    at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:3329)
    at 
org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1447)
    at 
org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1420)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:829)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1026)
    at org.codehaus.janino.UnitCompiler.access$700(UnitCompiler.java:236)
    at 
org.codehaus.janino.UnitCompiler$3.visitMemberClassDeclaration(UnitCompiler.java:425)
    at 
org.codehaus.janino.UnitCompiler$3.visitMemberClassDeclaration(UnitCompiler.java:418)
    at org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1533)
    at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:418)
    at 
org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1397)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:864)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:442)
    at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:236)
    at 
org.codehaus.janino.UnitCompiler$3.visitPackageMemberClassDeclaration(UnitCompiler.java:422)
    at 
org.codehaus.janino.UnitCompiler$3.visitPackageMemberClassDeclaration(UnitCompiler.java:418)
    at 
org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1688)
    at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:418)
    at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:392)
    at org.codehaus.janino.UnitCompiler.access$000(UnitCompiler.java:236)
    at 
org.codehaus.janino.UnitCompiler$2.visitCompilationUnit(UnitCompiler.java:363)
    at 
org.codehaus.janino.UnitCompiler$2.visitCompilationUnit(UnitCompiler.java:361)
    at org.codehaus.janino.Java$CompilationUnit.accept(Java.java:371)
    at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:361)
    at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:264)
    at org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:294)
    at org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:288)
    at org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:267)
    at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:82)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.doCompile(CodeGenerator.scala:1497)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.$anonfun$cache$1(CodeGenerator.scala:1589)
    at 
org.apache.spark.util.NonFateSharingCache$$anon$1.load(NonFateSharingCache.scala:68)
    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.util.NonFateSharingLoadingCache.$anonfun$get$2(NonFateSharingCache.scala:94)
    at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
    at 
org.apache.spark.util.NonFateSharingLoadingCache.get(NonFateSharingCache.scala:94)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1444)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:205)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:39)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1369)
    at 
org.apache.spark.sql.catalyst.expressions.SafeProjection$.createCodeGeneratedObject(Projection.scala:171)
    at 
org.apache.spark.sql.catalyst.expressions.SafeProjection$.createCodeGeneratedObject(Projection.scala:168)
    at 
org.apache.spark.sql.catalyst.expressions.CodeGeneratorWithInterpretedFallback.createObject(CodeGeneratorWithInterpretedFallback.scala:50)
    at 
org.apache.spark.sql.catalyst.expressions.SafeProjection$.create(Projection.scala:194)
    at 
org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Deserializer.apply(ExpressionEncoder.scala:180)
    at 
org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Deserializer.apply(ExpressionEncoder.scala:173)
    at scala.collection.ArrayOps$.map$extension(ArrayOps.scala:929)
    at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4344)
    at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:3585)
    at org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4334)
    at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
    at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:4332)
    at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
    at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
    at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
    at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
    at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
    at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4332)
    at org.apache.spark.sql.Dataset.collect(Dataset.scala:3585)
    at $line14.$read$$iw.<init>(<console>:1)
    at $line14.$read.<init>(<console>:15)
    at $line14.$read$.<clinit>(<console>:1)
    at $line14.$eval$.$print$lzycompute(<synthetic>:6)
    at $line14.$eval$.$print(<synthetic>:5)
    at $line14.$eval.$print(<synthetic>)
    at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native 
Method)
    at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
    at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.base/java.lang.reflect.Method.invoke(Method.java:568)
    at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:670)
    at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:1020)
    at scala.tools.nsc.interpreter.IMain.$anonfun$doInterpret$1(IMain.scala:506)
    at 
scala.reflect.internal.util.ScalaClassLoader.asContext(ScalaClassLoader.scala:36)
    at 
scala.reflect.internal.util.ScalaClassLoader.asContext$(ScalaClassLoader.scala:116)
    at 
scala.reflect.internal.util.AbstractFileClassLoader.asContext(AbstractFileClassLoader.scala:43)
    at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:505)
    at scala.tools.nsc.interpreter.IMain.$anonfun$doInterpret$3(IMain.scala:519)
    at scala.tools.nsc.interpreter.IMain.doInterpret(IMain.scala:519)
    at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:503)
    at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:501)
    at scala.tools.nsc.interpreter.shell.ILoop.loop$1(ILoop.scala:878)
    at 
scala.tools.nsc.interpreter.shell.ILoop.interpretStartingWith(ILoop.scala:906)
    at scala.tools.nsc.interpreter.shell.ILoop.command(ILoop.scala:433)
    at scala.tools.nsc.interpreter.shell.ILoop.processLine(ILoop.scala:440)
    at scala.tools.nsc.interpreter.shell.ILoop.loop(ILoop.scala:458)
    at scala.tools.nsc.interpreter.shell.ILoop.run(ILoop.scala:968)
    at org.apache.spark.repl.Main$.doMain(Main.scala:84)
    at org.apache.spark.repl.Main$.main(Main.scala:59)
    at org.apache.spark.repl.Main.main(Main.scala)
    at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native 
Method)
    at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
    at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.base/java.lang.reflect.Method.invoke(Method.java:568)
    at 
org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
    at 
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1029)
    at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:194)
    at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:217)
    at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:91)
    at 
org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1120)
    at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1129)
    at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
23/11/07 09:44:40 INFO CodeGenerator: 
/* 001 */ public java.lang.Object generate(Object[] references) {
/* 002 */   return new SpecificSafeProjection(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificSafeProjection extends 
org.apache.spark.sql.catalyst.expressions.codegen.BaseProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private int value_MapObject_lambda_variable_1;
/* 010 */   private boolean globalIsNull_0;
/* 011 */
/* 012 */   public SpecificSafeProjection(Object[] references) {
/* 013 */     this.references = references;
/* 014 */     mutableRow = (InternalRow) references[references.length - 1];
/* 015 */
/* 016 */
/* 017 */   }
/* 018 */
/* 019 */   public void initialize(int partitionIndex) {
/* 020 */
/* 021 */   }
/* 022 */
/* 023 */   public java.lang.Object apply(java.lang.Object _i) {
/* 024 */     InternalRow i = (InternalRow) _i;
/* 025 */     scala.collection.immutable.ArraySeq value_3 = MapObjects_0(i);
/* 026 */     if (globalIsNull_0) {
/* 027 */       mutableRow.setNullAt(0);
/* 028 */     } else {
/* 029 */
/* 030 */       mutableRow.update(0, value_3);
/* 031 */     }
/* 032 */
/* 033 */     return mutableRow;
/* 034 */   }
/* 035 */
/* 036 */
/* 037 */   private scala.collection.immutable.ArraySeq 
MapObjects_0(InternalRow i) {
/* 038 */     boolean isNull_1 = i.isNullAt(0);
/* 039 */     ArrayData value_1 = isNull_1 ?
/* 040 */     null : (i.getArray(0));
/* 041 */     scala.collection.immutable.ArraySeq value_0 = null;
/* 042 */
/* 043 */     if (!isNull_1) {
/* 044 */
/* 045 */       int dataLength_0 = value_1.numElements();
/* 046 */
/* 047 */       scala.collection.mutable.Builder collectionBuilder_0 = 
scala.collection.immutable.ArraySeq$.MODULE$.newBuilder();
/* 048 */       collectionBuilder_0.sizeHint(dataLength_0);
/* 049 */
/* 050 */
/* 051 */       int loopIndex_0 = 0;
/* 052 */
/* 053 */       while (loopIndex_0 < dataLength_0) {
/* 054 */         value_MapObject_lambda_variable_1 = (int) 
(value_1.getInt(loopIndex_0));
/* 055 */
/* 056 */
/* 057 */         if (false) {
/* 058 */           throw new NullPointerException(((java.lang.String) 
references[0] /* errMsg */));
/* 059 */         }
/* 060 */         if (false) {
/* 061 */           collectionBuilder_0.$plus$eq(null);
/* 062 */         } else {
/* 063 */           
collectionBuilder_0.$plus$eq(value_MapObject_lambda_variable_1);
/* 064 */         }
/* 065 */
/* 066 */         loopIndex_0 += 1;
/* 067 */       }
/* 068 */
/* 069 */       value_0 = (scala.collection.immutable.ArraySeq) 
collectionBuilder_0.result();
/* 070 */     }
/* 071 */     globalIsNull_0 = isNull_1;
/* 072 */     return value_0;
/* 073 */   }
/* 074 */
/* 075 */ }23/11/07 09:44:40 WARN SafeProjection: Expr codegen error and 
falling back to interpreter mode
java.util.concurrent.ExecutionException: 
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, 
Column 101: failed to compile: org.codehaus.commons.compiler.CompileException: 
File 'generated.java', Line 47, Column 101: No applicable constructor/method 
found for zero actual parameters; candidates are: "public 
scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(scala.reflect.ClassTag)", 
"public scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(java.lang.Object)", "public 
abstract scala.collection.mutable.Builder 
scala.collection.EvidenceIterableFactory.newBuilder(java.lang.Object)"
    at 
org.sparkproject.guava.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:306)
    at 
org.sparkproject.guava.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:293)
    at 
org.sparkproject.guava.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)
    at 
org.sparkproject.guava.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:135)
    at 
org.sparkproject.guava.cache.LocalCache$Segment.getAndRecordStats(LocalCache.java:2410)
    at 
org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2380)
    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.util.NonFateSharingLoadingCache.$anonfun$get$2(NonFateSharingCache.scala:94)
    at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
    at 
org.apache.spark.util.NonFateSharingLoadingCache.get(NonFateSharingCache.scala:94)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1444)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:205)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:39)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1369)
    at 
org.apache.spark.sql.catalyst.expressions.SafeProjection$.createCodeGeneratedObject(Projection.scala:171)
    at 
org.apache.spark.sql.catalyst.expressions.SafeProjection$.createCodeGeneratedObject(Projection.scala:168)
    at 
org.apache.spark.sql.catalyst.expressions.CodeGeneratorWithInterpretedFallback.createObject(CodeGeneratorWithInterpretedFallback.scala:50)
    at 
org.apache.spark.sql.catalyst.expressions.SafeProjection$.create(Projection.scala:194)
    at 
org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Deserializer.apply(ExpressionEncoder.scala:180)
    at 
org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Deserializer.apply(ExpressionEncoder.scala:173)
    at scala.collection.ArrayOps$.map$extension(ArrayOps.scala:929)
    at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4344)
    at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:3585)
    at org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4334)
    at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
    at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:4332)
    at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
    at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
    at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
    at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
    at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
    at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4332)
    at org.apache.spark.sql.Dataset.collect(Dataset.scala:3585)
    at $line14.$read$$iw.<init>(<console>:1)
    at $line14.$read.<init>(<console>:15)
    at $line14.$read$.<clinit>(<console>:1)
    at $line14.$eval$.$print$lzycompute(<synthetic>:6)
    at $line14.$eval$.$print(<synthetic>:5)
    at $line14.$eval.$print(<synthetic>)
    at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native 
Method)
    at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
    at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.base/java.lang.reflect.Method.invoke(Method.java:568)
    at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:670)
    at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:1020)
    at scala.tools.nsc.interpreter.IMain.$anonfun$doInterpret$1(IMain.scala:506)
    at 
scala.reflect.internal.util.ScalaClassLoader.asContext(ScalaClassLoader.scala:36)
    at 
scala.reflect.internal.util.ScalaClassLoader.asContext$(ScalaClassLoader.scala:116)
    at 
scala.reflect.internal.util.AbstractFileClassLoader.asContext(AbstractFileClassLoader.scala:43)
    at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:505)
    at scala.tools.nsc.interpreter.IMain.$anonfun$doInterpret$3(IMain.scala:519)
    at scala.tools.nsc.interpreter.IMain.doInterpret(IMain.scala:519)
    at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:503)
    at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:501)
    at scala.tools.nsc.interpreter.shell.ILoop.loop$1(ILoop.scala:878)
    at 
scala.tools.nsc.interpreter.shell.ILoop.interpretStartingWith(ILoop.scala:906)
    at scala.tools.nsc.interpreter.shell.ILoop.command(ILoop.scala:433)
    at scala.tools.nsc.interpreter.shell.ILoop.processLine(ILoop.scala:440)
    at scala.tools.nsc.interpreter.shell.ILoop.loop(ILoop.scala:458)
    at scala.tools.nsc.interpreter.shell.ILoop.run(ILoop.scala:968)
    at org.apache.spark.repl.Main$.doMain(Main.scala:84)
    at org.apache.spark.repl.Main$.main(Main.scala:59)
    at org.apache.spark.repl.Main.main(Main.scala)
    at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native 
Method)
    at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
    at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.base/java.lang.reflect.Method.invoke(Method.java:568)
    at 
org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
    at 
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1029)
    at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:194)
    at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:217)
    at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:91)
    at 
org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1120)
    at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1129)
    at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: org.codehaus.commons.compiler.CompileException: File 
'generated.java', Line 47, Column 101: failed to compile: 
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, 
Column 101: No applicable constructor/method found for zero actual parameters; 
candidates are: "public scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(scala.reflect.ClassTag)", 
"public scala.collection.mutable.Builder 
scala.collection.immutable.ArraySeq$.newBuilder(java.lang.Object)", "public 
abstract scala.collection.mutable.Builder 
scala.collection.EvidenceIterableFactory.newBuilder(java.lang.Object)"
    at 
org.apache.spark.sql.errors.QueryExecutionErrors$.compilerError(QueryExecutionErrors.scala:663)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.doCompile(CodeGenerator.scala:1509)
    at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.$anonfun$cache$1(CodeGenerator.scala:1589)
    at 
org.apache.spark.util.NonFateSharingCache$$anon$1.load(NonFateSharingCache.scala:68)
    at 
org.sparkproject.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
    at 
org.sparkproject.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
    ... 73 more
java.lang.ArrayStoreException: scala.collection.immutable.$colon$colon
  at scala.runtime.ScalaRunTime$.array_update(ScalaRunTime.scala:74)
  at scala.collection.ArrayOps$.map$extension(ArrayOps.scala:929)
  at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4344)
  at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:3585)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4334)
  at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:4332)
  at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
  at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
  at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
  at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4332)
  at org.apache.spark.sql.Dataset.collect(Dataset.scala:3585)
  ... 42 elided
 {code}



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

Reply via email to