Re: Spark 2.0.0 JaninoRuntimeException

2016-08-16 Thread Aris
Hello Michael,

I made a JIRA with sample code to reproduce this problem. I set you as the
"shepherd" -- I Hope this is enough, otherwise I can fix it.

https://issues.apache.org/jira/browse/SPARK-17092



On Sun, Aug 14, 2016 at 9:38 AM, Michael Armbrust 
wrote:

> Anytime you see JaninoRuntimeException you are seeing a bug in our code
> generation.  If you can come up with a small example that causes the
> problem it would be very helpful if you could open a JIRA.
>
> On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar 
> wrote:
>
>> I see a similar issue being resolved recently: https://issues.apach
>> e.org/jira/browse/SPARK-15285
>>
>> On Fri, Aug 12, 2016 at 3:33 PM, Aris  wrote:
>>
>>> Hello folks,
>>>
>>> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
>>> smaller data unit tests work on my laptop, when I'm on a cluster, I get
>>> cryptic error messages:
>>>
>>> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
 "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/spa
 rk/sql/catalyst/InternalRow;)I" of class "org.apache.spark.sql.catalyst
 .expressions.GeneratedClass$SpecificOrdering" grows beyond 64 KB

>>>
>>> Unfortunately I'm not clear on how to even isolate the source of this
>>> problem. I didn't have this problem in Spark 1.6.1.
>>>
>>> Any clues?
>>>
>>
>>
>>
>> --
>> -Dhruve Ashar
>>
>>
>


Re: Spark 2.0.0 JaninoRuntimeException

2016-08-16 Thread Aris
My error is specifically:

Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/spark/sql/catalyst/InternalRow;)I"
> of class "org.apache.spark.sql.catalyst.ex
> pressions.GeneratedClass$SpecificOrdering" grows beyond 64 KB


Here is an easy way to reproduce in spark-shell on a cluster!

import org.apache.spark.sql.types.{DoubleType, StructType}
import org.apache.spark.sql.{Row, SparkSession}

//val spark = SparkSession.builder.getOrCreate

val COLMAX: Double = 1000.0
val ROWSIZE: Int = 1000

val intToRow: Int => Row = (i: Int) =>
Row.fromSeq(Range.Double.inclusive(1.0, COLMAX, 1.0).toSeq)
val schema: StructType = (1 to COLMAX.toInt).foldLeft(new
StructType())((s, i) => s.add(i.toString, DoubleType, nullable =
true))
val rdds = spark.sparkContext.parallelize((1 to ROWSIZE).map(intToRow))
val df = spark.createDataFrame(rdds, schema)
val Array(left, right) = df.randomSplit(Array(.8,.2))

// This crashes
left.count




On Tue, Aug 16, 2016 at 8:56 AM, Ted Yu <yuzhih...@gmail.com> wrote:

> Can you take a look at commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf ?
>
> There was a test:
> SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB
>
> See if it matches your use case.
>
> On Tue, Aug 16, 2016 at 8:41 AM, Aris <arisofala...@gmail.com> wrote:
>
>> I am still working on making a minimal test that I can share without my
>> work-specific code being in there. However, the problem occurs with a
>> dataframe with several hundred columns being asked to do a tension split.
>> Random split works with up to about 350 columns so far. It breaks in my
>> code with 600 columns, but it's a converted dataset of case classes to
>> dataframe. This is deterministically causing the error in Scala 2.11.
>>
>> Once I can get a deterministically breaking test without work code I will
>> try to file a Jira bug.
>>
>> On Tue, Aug 16, 2016, 04:17 Ted Yu <yuzhih...@gmail.com> wrote:
>>
>>> I think we should reopen it.
>>>
>>> On Aug 16, 2016, at 1:48 AM, Kazuaki Ishizaki <ishiz...@jp.ibm.com>
>>> wrote:
>>>
>>> I just realized it since it broken a build with Scala 2.10.
>>> https://github.com/apache/spark/commit/fa244e5a90690d6a31be5
>>> 0f2aa203ae1a2e9a1cf
>>>
>>> I can reproduce the problem in SPARK-15285 with master branch.
>>> Should we reopen SPARK-15285?
>>>
>>> Best Regards,
>>> Kazuaki Ishizaki,
>>>
>>>
>>>
>>> From:Ted Yu <yuzhih...@gmail.com>
>>> To:dhruve ashar <dhruveas...@gmail.com>
>>> Cc:Aris <arisofala...@gmail.com>, "user@spark.apache.org" <
>>> user@spark.apache.org>
>>> Date:2016/08/15 06:19
>>> Subject:Re: Spark 2.0.0 JaninoRuntimeException
>>> --
>>>
>>>
>>>
>>> Looks like the proposed fix was reverted:
>>>
>>> Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply
>>> method grows beyond 64 KB"
>>>
>>> This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf.
>>>
>>> Maybe this was fixed in some other JIRA ?
>>>
>>> On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar <*dhruveas...@gmail.com*
>>> <dhruveas...@gmail.com>> wrote:
>>> I see a similar issue being resolved recently:
>>> *https://issues.apache.org/jira/browse/SPARK-15285*
>>> <https://issues.apache.org/jira/browse/SPARK-15285>
>>>
>>> On Fri, Aug 12, 2016 at 3:33 PM, Aris <*arisofala...@gmail.com*
>>> <arisofala...@gmail.com>> wrote:
>>> Hello folks,
>>>
>>> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
>>> smaller data unit tests work on my laptop, when I'm on a cluster, I get
>>> cryptic error messages:
>>>
>>> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
>>> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/
>>> spark/sql/catalyst/InternalRow;)I" of class
>>> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering"
>>> grows beyond 64 KB
>>>
>>> Unfortunately I'm not clear on how to even isolate the source of this
>>> problem. I didn't have this problem in Spark 1.6.1.
>>>
>>> Any clues?
>>>
>>>
>>>
>>> --
>>> -Dhruve Ashar
>>>
>>>
>>>
>>>
>


Re: Spark 2.0.0 JaninoRuntimeException

2016-08-16 Thread Ted Yu
Can you take a look at commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf ?

There was a test:
SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB

See if it matches your use case.

On Tue, Aug 16, 2016 at 8:41 AM, Aris <arisofala...@gmail.com> wrote:

> I am still working on making a minimal test that I can share without my
> work-specific code being in there. However, the problem occurs with a
> dataframe with several hundred columns being asked to do a tension split.
> Random split works with up to about 350 columns so far. It breaks in my
> code with 600 columns, but it's a converted dataset of case classes to
> dataframe. This is deterministically causing the error in Scala 2.11.
>
> Once I can get a deterministically breaking test without work code I will
> try to file a Jira bug.
>
> On Tue, Aug 16, 2016, 04:17 Ted Yu <yuzhih...@gmail.com> wrote:
>
>> I think we should reopen it.
>>
>> On Aug 16, 2016, at 1:48 AM, Kazuaki Ishizaki <ishiz...@jp.ibm.com>
>> wrote:
>>
>> I just realized it since it broken a build with Scala 2.10.
>> https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203a
>> e1a2e9a1cf
>>
>> I can reproduce the problem in SPARK-15285 with master branch.
>> Should we reopen SPARK-15285?
>>
>> Best Regards,
>> Kazuaki Ishizaki,
>>
>>
>>
>> From:Ted Yu <yuzhih...@gmail.com>
>> To:dhruve ashar <dhruveas...@gmail.com>
>> Cc:Aris <arisofala...@gmail.com>, "user@spark.apache.org" <
>> user@spark.apache.org>
>> Date:2016/08/15 06:19
>> Subject:Re: Spark 2.0.0 JaninoRuntimeException
>> --
>>
>>
>>
>> Looks like the proposed fix was reverted:
>>
>> Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply
>> method grows beyond 64 KB"
>>
>> This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf.
>>
>> Maybe this was fixed in some other JIRA ?
>>
>> On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar <*dhruveas...@gmail.com*
>> <dhruveas...@gmail.com>> wrote:
>> I see a similar issue being resolved recently:
>> *https://issues.apache.org/jira/browse/SPARK-15285*
>> <https://issues.apache.org/jira/browse/SPARK-15285>
>>
>> On Fri, Aug 12, 2016 at 3:33 PM, Aris <*arisofala...@gmail.com*
>> <arisofala...@gmail.com>> wrote:
>> Hello folks,
>>
>> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
>> smaller data unit tests work on my laptop, when I'm on a cluster, I get
>> cryptic error messages:
>>
>> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
>> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/
>> apache/spark/sql/catalyst/InternalRow;)I" of class "org.apache.spark.sql.
>> catalyst.expressions.GeneratedClass$SpecificOrdering" grows beyond 64 KB
>>
>> Unfortunately I'm not clear on how to even isolate the source of this
>> problem. I didn't have this problem in Spark 1.6.1.
>>
>> Any clues?
>>
>>
>>
>> --
>> -Dhruve Ashar
>>
>>
>>
>>


Re: Spark 2.0.0 JaninoRuntimeException

2016-08-16 Thread Aris
I am still working on making a minimal test that I can share without my
work-specific code being in there. However, the problem occurs with a
dataframe with several hundred columns being asked to do a tension split.
Random split works with up to about 350 columns so far. It breaks in my
code with 600 columns, but it's a converted dataset of case classes to
dataframe. This is deterministically causing the error in Scala 2.11.

Once I can get a deterministically breaking test without work code I will
try to file a Jira bug.

On Tue, Aug 16, 2016, 04:17 Ted Yu <yuzhih...@gmail.com> wrote:

> I think we should reopen it.
>
> On Aug 16, 2016, at 1:48 AM, Kazuaki Ishizaki <ishiz...@jp.ibm.com> wrote:
>
> I just realized it since it broken a build with Scala 2.10.
>
> https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf
>
> I can reproduce the problem in SPARK-15285 with master branch.
> Should we reopen SPARK-15285?
>
> Best Regards,
> Kazuaki Ishizaki,
>
>
>
> From:Ted Yu <yuzhih...@gmail.com>
> To:dhruve ashar <dhruveas...@gmail.com>
> Cc:Aris <arisofala...@gmail.com>, "user@spark.apache.org" <
> user@spark.apache.org>
> Date:2016/08/15 06:19
> Subject:Re: Spark 2.0.0 JaninoRuntimeException
> --
>
>
>
> Looks like the proposed fix was reverted:
>
> Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply
> method grows beyond 64 KB"
>
> This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf.
>
> Maybe this was fixed in some other JIRA ?
>
> On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar <*dhruveas...@gmail.com*
> <dhruveas...@gmail.com>> wrote:
> I see a similar issue being resolved recently:
> *https://issues.apache.org/jira/browse/SPARK-15285*
> <https://issues.apache.org/jira/browse/SPARK-15285>
>
> On Fri, Aug 12, 2016 at 3:33 PM, Aris <*arisofala...@gmail.com*
> <arisofala...@gmail.com>> wrote:
> Hello folks,
>
> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
> smaller data unit tests work on my laptop, when I'm on a cluster, I get
> cryptic error messages:
>
> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/spark/sql/catalyst/InternalRow;)I"
> of class
> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering"
> grows beyond 64 KB
>
> Unfortunately I'm not clear on how to even isolate the source of this
> problem. I didn't have this problem in Spark 1.6.1.
>
> Any clues?
>
>
>
> --
> -Dhruve Ashar
>
>
>
>


Re: Spark 2.0.0 JaninoRuntimeException

2016-08-16 Thread Ted Yu
I think we should reopen it. 

> On Aug 16, 2016, at 1:48 AM, Kazuaki Ishizaki <ishiz...@jp.ibm.com> wrote:
> 
> I just realized it since it broken a build with Scala 2.10.
> https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf
> 
> I can reproduce the problem in SPARK-15285 with master branch.
> Should we reopen SPARK-15285?
> 
> Best Regards,
> Kazuaki Ishizaki,
> 
> 
> 
> From:Ted Yu <yuzhih...@gmail.com>
> To:dhruve ashar <dhruveas...@gmail.com>
> Cc:Aris <arisofala...@gmail.com>, "user@spark.apache.org" 
> <user@spark.apache.org>
> Date:2016/08/15 06:19
> Subject:Re: Spark 2.0.0 JaninoRuntimeException
> 
> 
> 
> Looks like the proposed fix was reverted:
> 
> Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method 
> grows beyond 64 KB"
> 
> This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf.
> 
> Maybe this was fixed in some other JIRA ?
> 
> On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar <dhruveas...@gmail.com> wrote:
> I see a similar issue being resolved recently: 
> https://issues.apache.org/jira/browse/SPARK-15285
> 
> On Fri, Aug 12, 2016 at 3:33 PM, Aris <arisofala...@gmail.com> wrote:
> Hello folks,
> 
> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that smaller 
> data unit tests work on my laptop, when I'm on a cluster, I get cryptic error 
> messages:
> 
> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method 
> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/spark/sql/catalyst/InternalRow;)I"
>  of class 
> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering" 
> grows beyond 64 KB
> 
> Unfortunately I'm not clear on how to even isolate the source of this 
> problem. I didn't have this problem in Spark 1.6.1.
> 
> Any clues? 
> 
> 
> 
> -- 
> -Dhruve Ashar
> 
> 
> 


Re: Spark 2.0.0 JaninoRuntimeException

2016-08-16 Thread Kazuaki Ishizaki
I just realized it since it broken a build with Scala 2.10.
https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf

I can reproduce the problem in SPARK-15285 with master branch.
Should we reopen SPARK-15285?

Best Regards,
Kazuaki Ishizaki,



From:   Ted Yu <yuzhih...@gmail.com>
To: dhruve ashar <dhruveas...@gmail.com>
Cc: Aris <arisofala...@gmail.com>, "user@spark.apache.org" 
<user@spark.apache.org>
Date:   2016/08/15 06:19
Subject:Re: Spark 2.0.0 JaninoRuntimeException



Looks like the proposed fix was reverted:

Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply 
method grows beyond 64 KB"

This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf.

Maybe this was fixed in some other JIRA ?

On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar <dhruveas...@gmail.com> 
wrote:
I see a similar issue being resolved recently: 
https://issues.apache.org/jira/browse/SPARK-15285

On Fri, Aug 12, 2016 at 3:33 PM, Aris <arisofala...@gmail.com> wrote:
Hello folks,

I'm on Spark 2.0.0 working with Datasets -- and despite the fact that 
smaller data unit tests work on my laptop, when I'm on a cluster, I get 
cryptic error messages:

Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method 
"(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/spark/sql/catalyst/InternalRow;)I"
 
of class 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering" 
grows beyond 64 KB

Unfortunately I'm not clear on how to even isolate the source of this 
problem. I didn't have this problem in Spark 1.6.1.

Any clues? 



-- 
-Dhruve Ashar






Re: Spark 2.0.0 JaninoRuntimeException

2016-08-14 Thread Ted Yu
Looks like the proposed fix was reverted:

Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply
method grows beyond 64 KB"

This reverts commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf.

Maybe this was fixed in some other JIRA ?

On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar  wrote:

> I see a similar issue being resolved recently: https://issues.
> apache.org/jira/browse/SPARK-15285
>
> On Fri, Aug 12, 2016 at 3:33 PM, Aris  wrote:
>
>> Hello folks,
>>
>> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
>> smaller data unit tests work on my laptop, when I'm on a cluster, I get
>> cryptic error messages:
>>
>> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
>>> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/
>>> spark/sql/catalyst/InternalRow;)I" of class
>>> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering"
>>> grows beyond 64 KB
>>>
>>
>> Unfortunately I'm not clear on how to even isolate the source of this
>> problem. I didn't have this problem in Spark 1.6.1.
>>
>> Any clues?
>>
>
>
>
> --
> -Dhruve Ashar
>
>


Re: Spark 2.0.0 JaninoRuntimeException

2016-08-14 Thread Michael Armbrust
Anytime you see JaninoRuntimeException you are seeing a bug in our code
generation.  If you can come up with a small example that causes the
problem it would be very helpful if you could open a JIRA.

On Fri, Aug 12, 2016 at 2:30 PM, dhruve ashar  wrote:

> I see a similar issue being resolved recently: https://issues.
> apache.org/jira/browse/SPARK-15285
>
> On Fri, Aug 12, 2016 at 3:33 PM, Aris  wrote:
>
>> Hello folks,
>>
>> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
>> smaller data unit tests work on my laptop, when I'm on a cluster, I get
>> cryptic error messages:
>>
>> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
>>> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/
>>> spark/sql/catalyst/InternalRow;)I" of class
>>> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering"
>>> grows beyond 64 KB
>>>
>>
>> Unfortunately I'm not clear on how to even isolate the source of this
>> problem. I didn't have this problem in Spark 1.6.1.
>>
>> Any clues?
>>
>
>
>
> --
> -Dhruve Ashar
>
>


Re: Spark 2.0.0 JaninoRuntimeException

2016-08-12 Thread dhruve ashar
I see a similar issue being resolved recently:
https://issues.apache.org/jira/browse/SPARK-15285

On Fri, Aug 12, 2016 at 3:33 PM, Aris  wrote:

> Hello folks,
>
> I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
> smaller data unit tests work on my laptop, when I'm on a cluster, I get
> cryptic error messages:
>
> Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
>> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/
>> apache/spark/sql/catalyst/InternalRow;)I" of class "org.apache.spark.sql.
>> catalyst.expressions.GeneratedClass$SpecificOrdering" grows beyond 64 KB
>>
>
> Unfortunately I'm not clear on how to even isolate the source of this
> problem. I didn't have this problem in Spark 1.6.1.
>
> Any clues?
>



-- 
-Dhruve Ashar


Spark 2.0.0 JaninoRuntimeException

2016-08-12 Thread Aris
Hello folks,

I'm on Spark 2.0.0 working with Datasets -- and despite the fact that
smaller data unit tests work on my laptop, when I'm on a cluster, I get
cryptic error messages:

Caused by: org.codehaus.janino.JaninoRuntimeException: Code of method
> "(Lorg/apache/spark/sql/catalyst/InternalRow;Lorg/apache/spark/sql/catalyst/InternalRow;)I"
> of class
> "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering"
> grows beyond 64 KB
>

Unfortunately I'm not clear on how to even isolate the source of this
problem. I didn't have this problem in Spark 1.6.1.

Any clues?