[ 
https://issues.apache.org/jira/browse/SPARK-1520?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Patrick Wendell updated SPARK-1520:
-----------------------------------

    Description: 
This is a real doozie - when compiling a Spark assembly with JDK7, the produced 
jar does not work well with JRE6. I confirmed the byte code being produced is 
JDK 6 compatible (major version 50). What happens is that, silently, the JRE 
will not load any class files from the assembled jar.

{code}
$> sbt/sbt assembly/assembly

$> /usr/lib/jvm/java-1.7.0-openjdk-amd64/bin/java -cp 
/home/patrick/Documents/spark/assembly/target/scala-2.10/spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
 org.apache.spark.ui.UIWorkloadGenerator
usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] 
[FIFO|FAIR]

$> /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
/home/patrick/Documents/spark/assembly/target/scala-2.10/spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
 org.apache.spark.ui.UIWorkloadGenerator
Exception in thread "main" java.lang.NoClassDefFoundError: 
org/apache/spark/ui/UIWorkloadGenerator
Caused by: java.lang.ClassNotFoundException: 
org.apache.spark.ui.UIWorkloadGenerator
        at java.net.URLClassLoader$1.run(URLClassLoader.java:217)
        at java.security.AccessController.doPrivileged(Native Method)
        at java.net.URLClassLoader.findClass(URLClassLoader.java:205)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:323)
        at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:294)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:268)
Could not find the main class: org.apache.spark.ui.UIWorkloadGenerator. Program 
will exit.

{code}

I also noticed that if the jar is unzipped, and the classpath set to the 
currently directory, it "just works". Finally, if the assembly jar is compiled 
with JDK6, it also works. The error is seen with any class, not just the 
UIWorkloadGenerator. Also, this error doesn't exist in branch 0.9, only in 
master.

h1. Isolation and Cause

The package-time behavior of Java 6 and 7 differ with respect to the format 
used for jar files:
||Number of entries||JDK 6||JDK 7||
|<= 65536|zip|zip|
|> 65536|zip*|zip64|

zip* is a workaround for the original zip format that [described in 
JDK-6828461|https://bugs.openjdk.java.net/browse/JDK-4828461] that allows some 
versions of Java 6 to support larger assembly jars.

The Scala libraries we depend on have added a large number of classes which 
bumped us over the limit. This causes the Java 7 packaging to not work with 
Java 6. We can probably go back under the limit by clearing out some accidental 
inclusion of FastUtil, but eventually we'll go over again.

The real answer is to force people to build with JDK 6 if they want to run 
Spark on JRE 6.

-I've found that if I just unpack and re-pack the jar (using `jar`) it always 
works:-

{code}
$ cd assembly/target/scala-2.10/
$ /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
./spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar 
org.apache.spark.ui.UIWorkloadGenerator # fails
$ jar xvf spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
$ jar cvf spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar *
$ /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
./spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar 
org.apache.spark.ui.UIWorkloadGenerator # succeeds
{code}

-I also noticed something of note. The Breeze package contains single 
directories that have huge numbers of files in them (e.g. 2000+ class files in 
one directory). It's possible we are hitting some weird bugs/corner cases with 
compatibility of the internal storage format of the jar itself.-

-I narrowed this down specifically to the inclusion of the breeze library. Just 
adding breeze to an older (unaffected) build triggered the issue.-

-I ran a git bisection and this appeared after the MLLib sparse vector patch 
was merged:-
https://github.com/apache/spark/commit/80c29689ae3b589254a571da3ddb5f9c866ae534
SPARK-1212

  was:
This is a real doozie - when compiling a Spark assembly with JDK7, the produced 
jar does not work well with JRE6. I confirmed the byte code being produced is 
JDK 6 compatible (major version 50). What happens is that, silently, the JRE 
will not load any class files from the assembled jar.

{code}
$> sbt/sbt assembly/assembly

$> /usr/lib/jvm/java-1.7.0-openjdk-amd64/bin/java -cp 
/home/patrick/Documents/spark/assembly/target/scala-2.10/spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
 org.apache.spark.ui.UIWorkloadGenerator
usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] 
[FIFO|FAIR]

$> /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
/home/patrick/Documents/spark/assembly/target/scala-2.10/spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
 org.apache.spark.ui.UIWorkloadGenerator
Exception in thread "main" java.lang.NoClassDefFoundError: 
org/apache/spark/ui/UIWorkloadGenerator
Caused by: java.lang.ClassNotFoundException: 
org.apache.spark.ui.UIWorkloadGenerator
        at java.net.URLClassLoader$1.run(URLClassLoader.java:217)
        at java.security.AccessController.doPrivileged(Native Method)
        at java.net.URLClassLoader.findClass(URLClassLoader.java:205)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:323)
        at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:294)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:268)
Could not find the main class: org.apache.spark.ui.UIWorkloadGenerator. Program 
will exit.

{code}

I also noticed that if the jar is unzipped, and the classpath set to the 
currently directory, it "just works". Finally, if the assembly jar is compiled 
with JDK6, it also works. The error is seen with any class, not just the 
UIWorkloadGenerator. Also, this error doesn't exist in branch 0.9, only in 
master.

h1. Isolation and Cause

The package-time behavior of Java 6 and 7 differ with respect to the format 
used for jar files:
||Number of entries||JDK 6||JDK 7||
|<= 65536|zip|zip|
|> 65536|zip*|zip64|

zip* is a workaround for the original zip format that [described in 
JDK-6828461|https://bugs.openjdk.java.net/browse/JDK-4828461)] that allows some 
versions of Java 6 to support larger assembly jars.

The Scala libraries we depend on have added a large number of classes which 
bumped us over the limit. This causes the Java 7 packaging to not work with 
Java 6. We can probably go back under the limit by clearing out some accidental 
inclusion of FastUtil, but eventually we'll go over again.

The real answer is to force people to build with JDK 6 if they want to run 
Spark on JRE 6.

-I've found that if I just unpack and re-pack the jar (using `jar`) it always 
works:-

{code}
$ cd assembly/target/scala-2.10/
$ /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
./spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar 
org.apache.spark.ui.UIWorkloadGenerator # fails
$ jar xvf spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
$ jar cvf spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar *
$ /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
./spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar 
org.apache.spark.ui.UIWorkloadGenerator # succeeds
{code}

-I also noticed something of note. The Breeze package contains single 
directories that have huge numbers of files in them (e.g. 2000+ class files in 
one directory). It's possible we are hitting some weird bugs/corner cases with 
compatibility of the internal storage format of the jar itself.-

-I narrowed this down specifically to the inclusion of the breeze library. Just 
adding breeze to an older (unaffected) build triggered the issue.-

-I ran a git bisection and this appeared after the MLLib sparse vector patch 
was merged:-
https://github.com/apache/spark/commit/80c29689ae3b589254a571da3ddb5f9c866ae534
SPARK-1212


> Assembly Jar with more than 65536 files won't work when compiled on  JDK7 and 
> run on JDK6
> -----------------------------------------------------------------------------------------
>
>                 Key: SPARK-1520
>                 URL: https://issues.apache.org/jira/browse/SPARK-1520
>             Project: Spark
>          Issue Type: Bug
>          Components: MLlib, Spark Core
>            Reporter: Patrick Wendell
>            Priority: Blocker
>             Fix For: 1.0.0
>
>
> This is a real doozie - when compiling a Spark assembly with JDK7, the 
> produced jar does not work well with JRE6. I confirmed the byte code being 
> produced is JDK 6 compatible (major version 50). What happens is that, 
> silently, the JRE will not load any class files from the assembled jar.
> {code}
> $> sbt/sbt assembly/assembly
> $> /usr/lib/jvm/java-1.7.0-openjdk-amd64/bin/java -cp 
> /home/patrick/Documents/spark/assembly/target/scala-2.10/spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
>  org.apache.spark.ui.UIWorkloadGenerator
> usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] 
> [FIFO|FAIR]
> $> /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
> /home/patrick/Documents/spark/assembly/target/scala-2.10/spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
>  org.apache.spark.ui.UIWorkloadGenerator
> Exception in thread "main" java.lang.NoClassDefFoundError: 
> org/apache/spark/ui/UIWorkloadGenerator
> Caused by: java.lang.ClassNotFoundException: 
> org.apache.spark.ui.UIWorkloadGenerator
>       at java.net.URLClassLoader$1.run(URLClassLoader.java:217)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at java.net.URLClassLoader.findClass(URLClassLoader.java:205)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:323)
>       at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:294)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:268)
> Could not find the main class: org.apache.spark.ui.UIWorkloadGenerator. 
> Program will exit.
> {code}
> I also noticed that if the jar is unzipped, and the classpath set to the 
> currently directory, it "just works". Finally, if the assembly jar is 
> compiled with JDK6, it also works. The error is seen with any class, not just 
> the UIWorkloadGenerator. Also, this error doesn't exist in branch 0.9, only 
> in master.
> h1. Isolation and Cause
> The package-time behavior of Java 6 and 7 differ with respect to the format 
> used for jar files:
> ||Number of entries||JDK 6||JDK 7||
> |<= 65536|zip|zip|
> |> 65536|zip*|zip64|
> zip* is a workaround for the original zip format that [described in 
> JDK-6828461|https://bugs.openjdk.java.net/browse/JDK-4828461] that allows 
> some versions of Java 6 to support larger assembly jars.
> The Scala libraries we depend on have added a large number of classes which 
> bumped us over the limit. This causes the Java 7 packaging to not work with 
> Java 6. We can probably go back under the limit by clearing out some 
> accidental inclusion of FastUtil, but eventually we'll go over again.
> The real answer is to force people to build with JDK 6 if they want to run 
> Spark on JRE 6.
> -I've found that if I just unpack and re-pack the jar (using `jar`) it always 
> works:-
> {code}
> $ cd assembly/target/scala-2.10/
> $ /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
> ./spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar 
> org.apache.spark.ui.UIWorkloadGenerator # fails
> $ jar xvf spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar
> $ jar cvf spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar *
> $ /usr/lib/jvm/java-1.6.0-openjdk-amd64/bin/java -cp 
> ./spark-assembly-1.0.0-SNAPSHOT-hadoop1.0.4.jar 
> org.apache.spark.ui.UIWorkloadGenerator # succeeds
> {code}
> -I also noticed something of note. The Breeze package contains single 
> directories that have huge numbers of files in them (e.g. 2000+ class files 
> in one directory). It's possible we are hitting some weird bugs/corner cases 
> with compatibility of the internal storage format of the jar itself.-
> -I narrowed this down specifically to the inclusion of the breeze library. 
> Just adding breeze to an older (unaffected) build triggered the issue.-
> -I ran a git bisection and this appeared after the MLLib sparse vector patch 
> was merged:-
> https://github.com/apache/spark/commit/80c29689ae3b589254a571da3ddb5f9c866ae534
> SPARK-1212



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to