Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-13579][build][test-maven] Stop building the main Spark assembly. #11796

Closed
wants to merge 20 commits into from
Closed
Show file tree
Hide file tree
Changes from 17 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
101 changes: 31 additions & 70 deletions assembly/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -33,9 +33,8 @@

<properties>
<sbt.project.name>assembly</sbt.project.name>
<spark.jar.dir>scala-${scala.binary.version}</spark.jar.dir>
<spark.jar.basename>spark-assembly-${project.version}-hadoop${hadoop.version}.jar</spark.jar.basename>
<spark.jar>${project.build.directory}/${spark.jar.dir}/${spark.jar.basename}</spark.jar>
<build.testJarPhase>none</build.testJarPhase>
<build.copyDependenciesPhase>package</build.copyDependenciesPhase>
</properties>

<dependencies>
Expand Down Expand Up @@ -69,6 +68,17 @@
<artifactId>spark-repl_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>

<!--
Because we don't shade dependencies anymore, we need to restore Guava to compile scope so
that the libraries Spark depend on have it available. We'll package the version that Spark
uses (14.0.1) which is not the same as Hadoop dependencies, but works.
-->
<dependency>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When and if this turns out to be a problem for user applications, we can consider a different dependency shading strategy in a followup patch. Therefore, I'm fine with this change for now.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a problem for the Spark Cassandra Connector. The Cassandra Java Driver requires a 16.0 or greater version of guava. This necessarily means we need to shade now. This was on our roadmap anyway just wanted you to be aware.

<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<scope>${hadoop.deps.scope}</scope>
</dependency>
</dependencies>

<build>
Expand All @@ -87,75 +97,26 @@
<skip>true</skip>
</configuration>
</plugin>
<!-- zip pyspark archives to run python application on yarn mode -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
<configuration>
<target>
<delete dir="${basedir}/../python/lib/pyspark.zip"/>
<zip destfile="${basedir}/../python/lib/pyspark.zip">
<fileset dir="${basedir}/../python/" includes="pyspark/**/*"/>
</zip>
</target>
</configuration>
</plugin>
<!-- Use the shade plugin to create a big JAR with all the dependencies -->
<!-- zip pyspark archives to run python application on yarn mode -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<configuration>
<shadedArtifactAttached>false</shadedArtifactAttached>
<outputFile>${spark.jar}</outputFile>
<artifactSet>
<includes>
<include>*:*</include>
</includes>
</artifactSet>
<filters>
<filter>
<artifact>*:*</artifact>
<excludes>
<exclude>org/datanucleus/**</exclude>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
</excludes>
</filter>
</filters>
</configuration>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>META-INF/services/org.apache.hadoop.fs.FileSystem</resource>
</transformer>
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>reference.conf</resource>
</transformer>
<transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
<resource>log4j.properties</resource>
</transformer>
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/>
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
</transformers>
</configuration>
</execution>
</executions>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
<configuration>
<target>
<delete dir="${basedir}/../python/lib/pyspark.zip"/>
<zip destfile="${basedir}/../python/lib/pyspark.zip">
<fileset dir="${basedir}/../python/" includes="pyspark/**/*"/>
</zip>
</target>
</configuration>
</plugin>
</plugins>
</build>
Expand Down
9 changes: 4 additions & 5 deletions bin/spark-class
Original file line number Diff line number Diff line change
Expand Up @@ -36,21 +36,20 @@ else
fi

# Find Spark jars.
# TODO: change the directory name when Spark jars move from "lib".
if [ -f "${SPARK_HOME}/RELEASE" ]; then
SPARK_JARS_DIR="${SPARK_HOME}/lib"
SPARK_JARS_DIR="${SPARK_HOME}/jars"
else
SPARK_JARS_DIR="${SPARK_HOME}/assembly/target/scala-$SPARK_SCALA_VERSION"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you also have to append /jars here?

fi

if [ ! -d "$SPARK_JARS_DIR" ]; then
if [ ! -d "$SPARK_JARS_DIR" ] && [ -z "$SPARK_TESTING$SPARK_SQL_TESTING" ]; then
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this an idiomatic way to do an or check? Looked like a typo to me at first glance, but I now see what it's doing.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can add an explicit && [ -z ... ] if you prefer, just chose the shorter version in this case.

echo "Failed to find Spark jars directory ($SPARK_JARS_DIR)." 1>&2
echo "You need to build Spark before running this program." 1>&2
exit 1
else
LAUNCH_CLASSPATH="$SPARK_JARS_DIR/*"
fi

LAUNCH_CLASSPATH="$SPARK_JARS_DIR/*"

# Add the launcher build dir to the classpath if requested.
if [ -n "$SPARK_PREPEND_CLASSES" ]; then
LAUNCH_CLASSPATH="${SPARK_HOME}/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH"
Expand Down
3 changes: 1 addition & 2 deletions bin/spark-class2.cmd
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,8 @@ if "x%1"=="x" (
)

rem Find Spark jars.
rem TODO: change the directory name when Spark jars move from "lib".
if exist "%SPARK_HOME%\RELEASE" (
set SPARK_JARS_DIR="%SPARK_HOME%\lib"
set SPARK_JARS_DIR="%SPARK_HOME%\jars"
) else (
set SPARK_JARS_DIR="%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION%"
)
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1120,9 +1120,9 @@ private[spark] object Utils extends Logging {
extraEnvironment: Map[String, String] = Map.empty,
redirectStderr: Boolean = true): String = {
val process = executeCommand(command, workingDir, extraEnvironment, redirectStderr)
val output = new StringBuffer
val output = new StringBuilder
val threadName = "read stdout for " + command(0)
def appendToOutput(s: String): Unit = output.append(s)
def appendToOutput(s: String): Unit = output.append(s).append("\n")
val stdoutThread = processStreamByLine(threadName, process.getInputStream, appendToOutput)
val exitCode = process.waitFor()
stdoutThread.join() // Wait for it to finish reading output
Expand Down
72 changes: 41 additions & 31 deletions core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -201,24 +201,29 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging {

// Make sure only logging errors
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the deal with these logging changes? Got a short summary?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For some reason these tests failed in one of my runs. And when they failed, they used to leave the error level at "ERROR", which basically means all tests that ran after these didn't write any logs.

val logger = Logger.getRootLogger
val oldLogLevel = logger.getLevel
logger.setLevel(Level.ERROR)
logger.addAppender(mockAppender)
try {
logger.addAppender(mockAppender)

val testOutputStream = new PipedOutputStream()
val testInputStream = new PipedInputStream(testOutputStream)
val testOutputStream = new PipedOutputStream()
val testInputStream = new PipedInputStream(testOutputStream)

// Close the stream before appender tries to read will cause an IOException
testInputStream.close()
testOutputStream.close()
val appender = FileAppender(testInputStream, testFile, new SparkConf)
// Close the stream before appender tries to read will cause an IOException
testInputStream.close()
testOutputStream.close()
val appender = FileAppender(testInputStream, testFile, new SparkConf)

appender.awaitTermination()
appender.awaitTermination()

// If InputStream was closed without first stopping the appender, an exception will be logged
verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture)
val loggingEvent = loggingEventCaptor.getValue
assert(loggingEvent.getThrowableInformation !== null)
assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
// If InputStream was closed without first stopping the appender, an exception will be logged
verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture)
val loggingEvent = loggingEventCaptor.getValue
assert(loggingEvent.getThrowableInformation !== null)
assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
} finally {
logger.setLevel(oldLogLevel)
}
}

test("file appender async close stream gracefully") {
Expand All @@ -228,30 +233,35 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging {

// Make sure only logging errors
val logger = Logger.getRootLogger
val oldLogLevel = logger.getLevel
logger.setLevel(Level.ERROR)
logger.addAppender(mockAppender)
try {
logger.addAppender(mockAppender)

val testOutputStream = new PipedOutputStream()
val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream
val testOutputStream = new PipedOutputStream()
val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream

// Close the stream before appender tries to read will cause an IOException
testInputStream.close()
testOutputStream.close()
val appender = FileAppender(testInputStream, testFile, new SparkConf)
// Close the stream before appender tries to read will cause an IOException
testInputStream.close()
testOutputStream.close()
val appender = FileAppender(testInputStream, testFile, new SparkConf)

// Stop the appender before an IOException is called during read
testInputStream.latchReadStarted.await()
appender.stop()
testInputStream.latchReadProceed.countDown()
// Stop the appender before an IOException is called during read
testInputStream.latchReadStarted.await()
appender.stop()
testInputStream.latchReadProceed.countDown()

appender.awaitTermination()
appender.awaitTermination()

// Make sure no IOException errors have been logged as a result of appender closing gracefully
verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture)
import scala.collection.JavaConverters._
loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent =>
assert(loggingEvent.getThrowableInformation === null
|| !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
// Make sure no IOException errors have been logged as a result of appender closing gracefully
verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture)
import scala.collection.JavaConverters._
loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent =>
assert(loggingEvent.getThrowableInformation === null
|| !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
}
} finally {
logger.setLevel(oldLogLevel)
}
}

Expand Down
4 changes: 1 addition & 3 deletions dev/deps/spark-deps-hadoop-2.2
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ asm-3.1.jar
asm-commons-3.1.jar
asm-tree-3.1.jar
avro-1.7.7.jar
avro-ipc-1.7.7-tests.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
bonecp-0.8.0.RELEASE.jar
Expand Down Expand Up @@ -60,6 +59,7 @@ grizzly-http-2.1.2.jar
grizzly-http-server-2.1.2.jar
grizzly-http-servlet-2.1.2.jar
grizzly-rcm-2.1.2.jar
guava-14.0.1.jar
guice-3.0.jar
guice-servlet-3.0.jar
hadoop-annotations-2.2.0.jar
Expand Down Expand Up @@ -163,7 +163,6 @@ scala-parser-combinators_2.11-1.0.4.jar
scala-reflect-2.11.7.jar
scala-xml_2.11-1.0.2.jar
scalap-2.11.7.jar
servlet-api-2.5.jar
slf4j-api-1.7.16.jar
slf4j-log4j12-1.7.16.jar
snappy-0.2.jar
Expand All @@ -175,7 +174,6 @@ stax-api-1.0.1.jar
stream-2.7.0.jar
super-csv-2.2.0.jar
univocity-parsers-1.5.6.jar
unused-1.0.0.jar
xbean-asm5-shaded-4.4.jar
xmlenc-0.52.jar
xz-1.0.jar
Expand Down
4 changes: 1 addition & 3 deletions dev/deps/spark-deps-hadoop-2.3
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ asm-3.1.jar
asm-commons-3.1.jar
asm-tree-3.1.jar
avro-1.7.7.jar
avro-ipc-1.7.7-tests.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
base64-2.3.8.jar
Expand Down Expand Up @@ -55,6 +54,7 @@ eigenbase-properties-1.1.5.jar
geronimo-annotation_1.0_spec-1.1.1.jar
geronimo-jaspic_1.0_spec-1.0.jar
geronimo-jta_1.1_spec-1.1.1.jar
guava-14.0.1.jar
guice-3.0.jar
guice-servlet-3.0.jar
hadoop-annotations-2.3.0.jar
Expand Down Expand Up @@ -154,7 +154,6 @@ scala-parser-combinators_2.11-1.0.4.jar
scala-reflect-2.11.7.jar
scala-xml_2.11-1.0.2.jar
scalap-2.11.7.jar
servlet-api-2.5.jar
slf4j-api-1.7.16.jar
slf4j-log4j12-1.7.16.jar
snappy-0.2.jar
Expand All @@ -166,7 +165,6 @@ stax-api-1.0.1.jar
stream-2.7.0.jar
super-csv-2.2.0.jar
univocity-parsers-1.5.6.jar
unused-1.0.0.jar
xbean-asm5-shaded-4.4.jar
xmlenc-0.52.jar
xz-1.0.jar
Expand Down
4 changes: 1 addition & 3 deletions dev/deps/spark-deps-hadoop-2.4
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ asm-3.1.jar
asm-commons-3.1.jar
asm-tree-3.1.jar
avro-1.7.7.jar
avro-ipc-1.7.7-tests.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
base64-2.3.8.jar
Expand Down Expand Up @@ -55,6 +54,7 @@ eigenbase-properties-1.1.5.jar
geronimo-annotation_1.0_spec-1.1.1.jar
geronimo-jaspic_1.0_spec-1.0.jar
geronimo-jta_1.1_spec-1.1.1.jar
guava-14.0.1.jar
guice-3.0.jar
guice-servlet-3.0.jar
hadoop-annotations-2.4.0.jar
Expand Down Expand Up @@ -155,7 +155,6 @@ scala-parser-combinators_2.11-1.0.4.jar
scala-reflect-2.11.7.jar
scala-xml_2.11-1.0.2.jar
scalap-2.11.7.jar
servlet-api-2.5.jar
slf4j-api-1.7.16.jar
slf4j-log4j12-1.7.16.jar
snappy-0.2.jar
Expand All @@ -167,7 +166,6 @@ stax-api-1.0.1.jar
stream-2.7.0.jar
super-csv-2.2.0.jar
univocity-parsers-1.5.6.jar
unused-1.0.0.jar
xbean-asm5-shaded-4.4.jar
xmlenc-0.52.jar
xz-1.0.jar
Expand Down
4 changes: 1 addition & 3 deletions dev/deps/spark-deps-hadoop-2.6
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ asm-3.1.jar
asm-commons-3.1.jar
asm-tree-3.1.jar
avro-1.7.7.jar
avro-ipc-1.7.7-tests.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
base64-2.3.8.jar
Expand Down Expand Up @@ -60,6 +59,7 @@ geronimo-annotation_1.0_spec-1.1.1.jar
geronimo-jaspic_1.0_spec-1.0.jar
geronimo-jta_1.1_spec-1.1.1.jar
gson-2.2.4.jar
guava-14.0.1.jar
guice-3.0.jar
guice-servlet-3.0.jar
hadoop-annotations-2.6.0.jar
Expand Down Expand Up @@ -161,7 +161,6 @@ scala-parser-combinators_2.11-1.0.4.jar
scala-reflect-2.11.7.jar
scala-xml_2.11-1.0.2.jar
scalap-2.11.7.jar
servlet-api-2.5.jar
slf4j-api-1.7.16.jar
slf4j-log4j12-1.7.16.jar
snappy-0.2.jar
Expand All @@ -173,7 +172,6 @@ stax-api-1.0.1.jar
stream-2.7.0.jar
super-csv-2.2.0.jar
univocity-parsers-1.5.6.jar
unused-1.0.0.jar
xbean-asm5-shaded-4.4.jar
xercesImpl-2.9.1.jar
xmlenc-0.52.jar
Expand Down
Loading