[ https://issues.apache.org/jira/browse/SPARK-15094?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sandeep Singh updated SPARK-15094: ---------------------------------- Description: 2Steps to reproduce in console below. {code} scala> import scala.reflect.ClassTag import scala.reflect.ClassTag scala> import org.apache.spark.sql.Dataset import org.apache.spark.sql.Dataset scala> case class Generic[T](id: T, value: Double) defined class Generic scala> val ds = Seq(Generic(1, 1.0), Generic(2, 2.0)).toDS ds: org.apache.spark.sql.Dataset[Generic[Int]] = [id: int, value: double] scala> val ds2 = ds.map(g => Generic(g.id, g.value)) ds2: org.apache.spark.sql.Dataset[Generic[Int]] = [id: int, value: double] scala> def codeGenError[T : ClassTag](dataset: Dataset[Generic[T]]) = dataset.rdd.map(r => r.id).count codeGenError: [T](dataset: org.apache.spark.sql.Dataset[Generic[T]])(implicit evidence$1: scala.reflect.ClassTag[T])Long scala> codeGenError(ds) res0: Long = 2 scala> codeGenError(ds2) 16/05/03 20:57:01 ERROR CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 31, Column 29: No applicable constructor/method found for actual parameters "int, $line17.$read$$iw$$iw$Generic"; candidates are: "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, double)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, float)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, long)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.CalendarInterval)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[], int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[])", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.UTF8String)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.sql.types.Decimal, int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, short)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, boolean)" /* 001 */ /* 002 */ public java.lang.Object generate(Object[] references) { /* 003 */ return new SpecificUnsafeProjection(references); /* 004 */ } /* 005 */ /* 006 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection { /* 007 */ /* 008 */ private Object[] references; /* 009 */ private UnsafeRow result; /* 010 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder holder; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter rowWriter; /* 012 */ /* 013 */ /* 014 */ public SpecificUnsafeProjection(Object[] references) { /* 015 */ this.references = references; /* 016 */ result = new UnsafeRow(1); /* 017 */ this.holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(result, 32); /* 018 */ this.rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(holder, 1); /* 019 */ } /* 020 */ /* 021 */ // Scala.Function1 need this /* 022 */ public java.lang.Object apply(java.lang.Object row) { /* 023 */ return apply((InternalRow) row); /* 024 */ } /* 025 */ /* 026 */ public UnsafeRow apply(InternalRow i) { /* 027 */ holder.reset(); /* 028 */ /* 029 */ /* input[0, $line17.$read$$iw$$iw$Generic] */ /* 030 */ $line17.$read$$iw$$iw$Generic value = ($line17.$read$$iw$$iw$Generic)i.get(0, null); /* 031 */ rowWriter.write(0, value); /* 032 */ result.setTotalSize(holder.totalSize()); /* 033 */ return result; /* 034 */ } /* 035 */ } /* 036 */ org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 31, Column 29: No applicable constructor/method found for actual parameters "int, $line17.$read$$iw$$iw$Generic"; candidates are: "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, double)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, float)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, long)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.CalendarInterval)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[], int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[])", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.UTF8String)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.sql.types.Decimal, int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, short)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, boolean)" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:10174) at org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:7559) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7429) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7333) at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:3873) at org.codehaus.janino.UnitCompiler.access$6900(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$10.visitMethodInvocation(UnitCompiler.java:3263) at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:3974) at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:3290) at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:4368) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2662) at org.codehaus.janino.UnitCompiler.access$4400(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$7.visitMethodInvocation(UnitCompiler.java:2627) at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:3974) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2654) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1643) at org.codehaus.janino.UnitCompiler.access$1100(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$4.visitExpressionStatement(UnitCompiler.java:936) at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2097) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:958) at org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1007) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2293) at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:822) at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:794) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:507) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:658) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:662) at org.codehaus.janino.UnitCompiler.access$600(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:350) at org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1035) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:354) at org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:769) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:532) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:393) at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:347) at org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1139) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:354) at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:322) at org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:383) at org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:315) at org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:233) at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:192) at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:84) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:732) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:757) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:754) at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599) at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2379) at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342) at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2257) at com.google.common.cache.LocalCache.get(LocalCache.java:4000) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:696) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:395) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:352) at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:151) at org.apache.spark.sql.execution.ProjectExec$$anonfun$8.apply(basicPhysicalOperators.scala:68) at org.apache.spark.sql.execution.ProjectExec$$anonfun$8.apply(basicPhysicalOperators.scala:67) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$23.apply(RDD.scala:771) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$23.apply(RDD.scala:771) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:85) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:254) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) {code} was: Steps to reproduce in console below. {code} scala> import scala.reflect.ClassTag import scala.reflect.ClassTag scala> import org.apache.spark.sql.Dataset import org.apache.spark.sql.Dataset scala> case class Generic[T](id: T, value: Double) defined class Generic scala> val ds = Seq(Generic(1, 1.0), Generic(2, 2.0)).toDS ds: org.apache.spark.sql.Dataset[Generic[Int]] = [id: int, value: double] scala> val ds2 = ds.map(g => Generic(g.id, g.value)) ds2: org.apache.spark.sql.Dataset[Generic[Int]] = [id: int, value: double] scala> def codeGenError[T : ClassTag](dataset: Dataset[Generic[T]]) = dataset.rdd.map(r => r.id).count codeGenError: [T](dataset: org.apache.spark.sql.Dataset[Generic[T]])(implicit evidence$1: scala.reflect.ClassTag[T])Long scala> codeGenError(ds) res0: Long = 2 scala> codeGenError(ds2) 16/05/03 20:57:01 ERROR CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 31, Column 29: No applicable constructor/method found for actual parameters "int, $line17.$read$$iw$$iw$Generic"; candidates are: "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, double)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, float)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, long)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.CalendarInterval)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[], int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[])", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.UTF8String)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.sql.types.Decimal, int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, short)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, boolean)" /* 001 */ /* 002 */ public java.lang.Object generate(Object[] references) { /* 003 */ return new SpecificUnsafeProjection(references); /* 004 */ } /* 005 */ /* 006 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection { /* 007 */ /* 008 */ private Object[] references; /* 009 */ private UnsafeRow result; /* 010 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder holder; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter rowWriter; /* 012 */ /* 013 */ /* 014 */ public SpecificUnsafeProjection(Object[] references) { /* 015 */ this.references = references; /* 016 */ result = new UnsafeRow(1); /* 017 */ this.holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(result, 32); /* 018 */ this.rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(holder, 1); /* 019 */ } /* 020 */ /* 021 */ // Scala.Function1 need this /* 022 */ public java.lang.Object apply(java.lang.Object row) { /* 023 */ return apply((InternalRow) row); /* 024 */ } /* 025 */ /* 026 */ public UnsafeRow apply(InternalRow i) { /* 027 */ holder.reset(); /* 028 */ /* 029 */ /* input[0, $line17.$read$$iw$$iw$Generic] */ /* 030 */ $line17.$read$$iw$$iw$Generic value = ($line17.$read$$iw$$iw$Generic)i.get(0, null); /* 031 */ rowWriter.write(0, value); /* 032 */ result.setTotalSize(holder.totalSize()); /* 033 */ return result; /* 034 */ } /* 035 */ } /* 036 */ org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 31, Column 29: No applicable constructor/method found for actual parameters "int, $line17.$read$$iw$$iw$Generic"; candidates are: "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, double)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, float)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, long)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.CalendarInterval)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[], int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, byte[])", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.unsafe.types.UTF8String)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, org.apache.spark.sql.types.Decimal, int, int)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, short)", "public void org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, boolean)" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:10174) at org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:7559) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7429) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7333) at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:3873) at org.codehaus.janino.UnitCompiler.access$6900(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$10.visitMethodInvocation(UnitCompiler.java:3263) at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:3974) at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:3290) at org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:4368) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2662) at org.codehaus.janino.UnitCompiler.access$4400(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$7.visitMethodInvocation(UnitCompiler.java:2627) at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:3974) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2654) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1643) at org.codehaus.janino.UnitCompiler.access$1100(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$4.visitExpressionStatement(UnitCompiler.java:936) at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2097) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:958) at org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1007) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2293) at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:822) at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:794) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:507) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:658) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:662) at org.codehaus.janino.UnitCompiler.access$600(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:350) at org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1035) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:354) at org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:769) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:532) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:393) at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:185) at org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:347) at org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1139) at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:354) at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:322) at org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:383) at org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:315) at org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:233) at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:192) at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:84) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:732) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:757) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:754) at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599) at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2379) at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342) at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2257) at com.google.common.cache.LocalCache.get(LocalCache.java:4000) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:696) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:395) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:352) at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:151) at org.apache.spark.sql.execution.ProjectExec$$anonfun$8.apply(basicPhysicalOperators.scala:68) at org.apache.spark.sql.execution.ProjectExec$$anonfun$8.apply(basicPhysicalOperators.scala:67) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$23.apply(RDD.scala:771) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$23.apply(RDD.scala:771) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:85) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:254) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) {code} > CodeGenerator: failed to compile - when using dataset.rdd with generic case > class > --------------------------------------------------------------------------------- > > Key: SPARK-15094 > URL: https://issues.apache.org/jira/browse/SPARK-15094 > Project: Spark > Issue Type: Bug > Components: SQL > Reporter: Nick Pentreath > > 2Steps to reproduce in console below. > {code} > scala> import scala.reflect.ClassTag > import scala.reflect.ClassTag > scala> import org.apache.spark.sql.Dataset > import org.apache.spark.sql.Dataset > scala> case class Generic[T](id: T, value: Double) > defined class Generic > scala> val ds = Seq(Generic(1, 1.0), Generic(2, 2.0)).toDS > ds: org.apache.spark.sql.Dataset[Generic[Int]] = [id: int, value: double] > scala> val ds2 = ds.map(g => Generic(g.id, g.value)) > ds2: org.apache.spark.sql.Dataset[Generic[Int]] = [id: int, value: double] > scala> def codeGenError[T : ClassTag](dataset: Dataset[Generic[T]]) = > dataset.rdd.map(r => r.id).count > codeGenError: [T](dataset: org.apache.spark.sql.Dataset[Generic[T]])(implicit > evidence$1: scala.reflect.ClassTag[T])Long > scala> codeGenError(ds) > res0: Long = 2 > scala> codeGenError(ds2) > 16/05/03 20:57:01 ERROR CodeGenerator: failed to compile: > org.codehaus.commons.compiler.CompileException: File 'generated.java', Line > 31, Column 29: No applicable constructor/method found for actual parameters > "int, $line17.$read$$iw$$iw$Generic"; candidates are: "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > double)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > float)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > long)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > int)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > byte)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > org.apache.spark.unsafe.types.CalendarInterval)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > byte[], int, int)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > byte[])", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > org.apache.spark.unsafe.types.UTF8String)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > org.apache.spark.sql.types.Decimal, int, int)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > short)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > boolean)" > /* 001 */ > /* 002 */ public java.lang.Object generate(Object[] references) { > /* 003 */ return new SpecificUnsafeProjection(references); > /* 004 */ } > /* 005 */ > /* 006 */ class SpecificUnsafeProjection extends > org.apache.spark.sql.catalyst.expressions.UnsafeProjection { > /* 007 */ > /* 008 */ private Object[] references; > /* 009 */ private UnsafeRow result; > /* 010 */ private > org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder holder; > /* 011 */ private > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter rowWriter; > /* 012 */ > /* 013 */ > /* 014 */ public SpecificUnsafeProjection(Object[] references) { > /* 015 */ this.references = references; > /* 016 */ result = new UnsafeRow(1); > /* 017 */ this.holder = new > org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(result, 32); > /* 018 */ this.rowWriter = new > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(holder, 1); > /* 019 */ } > /* 020 */ > /* 021 */ // Scala.Function1 need this > /* 022 */ public java.lang.Object apply(java.lang.Object row) { > /* 023 */ return apply((InternalRow) row); > /* 024 */ } > /* 025 */ > /* 026 */ public UnsafeRow apply(InternalRow i) { > /* 027 */ holder.reset(); > /* 028 */ > /* 029 */ /* input[0, $line17.$read$$iw$$iw$Generic] */ > /* 030 */ $line17.$read$$iw$$iw$Generic value = > ($line17.$read$$iw$$iw$Generic)i.get(0, null); > /* 031 */ rowWriter.write(0, value); > /* 032 */ result.setTotalSize(holder.totalSize()); > /* 033 */ return result; > /* 034 */ } > /* 035 */ } > /* 036 */ > org.codehaus.commons.compiler.CompileException: File 'generated.java', Line > 31, Column 29: No applicable constructor/method found for actual parameters > "int, $line17.$read$$iw$$iw$Generic"; candidates are: "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > double)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > float)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > long)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > int)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > byte)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > org.apache.spark.unsafe.types.CalendarInterval)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > byte[], int, int)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > byte[])", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > org.apache.spark.unsafe.types.UTF8String)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > org.apache.spark.sql.types.Decimal, int, int)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > short)", "public void > org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(int, > boolean)" > at > org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:10174) > at > org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:7559) > at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7429) > at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:7333) > at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:3873) > at org.codehaus.janino.UnitCompiler.access$6900(UnitCompiler.java:185) > at > org.codehaus.janino.UnitCompiler$10.visitMethodInvocation(UnitCompiler.java:3263) > at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:3974) > at org.codehaus.janino.UnitCompiler.compileGet(UnitCompiler.java:3290) > at > org.codehaus.janino.UnitCompiler.compileGetValue(UnitCompiler.java:4368) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:2662) > at org.codehaus.janino.UnitCompiler.access$4400(UnitCompiler.java:185) > at > org.codehaus.janino.UnitCompiler$7.visitMethodInvocation(UnitCompiler.java:2627) > at org.codehaus.janino.Java$MethodInvocation.accept(Java.java:3974) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2654) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:1643) > at org.codehaus.janino.UnitCompiler.access$1100(UnitCompiler.java:185) > at > org.codehaus.janino.UnitCompiler$4.visitExpressionStatement(UnitCompiler.java:936) > at org.codehaus.janino.Java$ExpressionStatement.accept(Java.java:2097) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:958) > at > org.codehaus.janino.UnitCompiler.compileStatements(UnitCompiler.java:1007) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2293) > at > org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:822) > at > org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:794) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:507) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:658) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:662) > at org.codehaus.janino.UnitCompiler.access$600(UnitCompiler.java:185) > at > org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:350) > at > org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1035) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:354) > at > org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:769) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:532) > at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:393) > at org.codehaus.janino.UnitCompiler.access$400(UnitCompiler.java:185) > at > org.codehaus.janino.UnitCompiler$2.visitPackageMemberClassDeclaration(UnitCompiler.java:347) > at > org.codehaus.janino.Java$PackageMemberClassDeclaration.accept(Java.java:1139) > at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:354) > at org.codehaus.janino.UnitCompiler.compileUnit(UnitCompiler.java:322) > at > org.codehaus.janino.SimpleCompiler.compileToClassLoader(SimpleCompiler.java:383) > at > org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:315) > at > org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:233) > at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:192) > at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:84) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:732) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:757) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:754) > at > com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599) > at > com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2379) > at > com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342) > at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2257) > at com.google.common.cache.LocalCache.get(LocalCache.java:4000) > at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) > at > com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:696) > at > org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:395) > at > org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:352) > at > org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:151) > at > org.apache.spark.sql.execution.ProjectExec$$anonfun$8.apply(basicPhysicalOperators.scala:68) > at > org.apache.spark.sql.execution.ProjectExec$$anonfun$8.apply(basicPhysicalOperators.scala:67) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$23.apply(RDD.scala:771) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$23.apply(RDD.scala:771) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) > at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) > at org.apache.spark.scheduler.Task.run(Task.scala:85) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:254) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org