This reopens PR 649 from incubator-spark against the new repo
Author: Sandy Ryza <sandy@cloudera.com>
Closes#102 from sryza/sandy-spark-1064 and squashes the following commits:
270e490 [Sandy Ryza] Handle different application classpath variables in different versions
88b04e0 [Sandy Ryza] SPARK-1064. Make it possible to run on YARN without bundling Hadoop jars in Spark assembly
This patch removes Ganglia integration from the default build. It
allows users willing to link against LGPL code to use Ganglia
by adding build flags or linking against a new Spark artifact called
spark-ganglia-lgpl.
This brings Spark in line with the Apache policy on LGPL code
enumerated here:
https://www.apache.org/legal/3party.html#options-optional
Author: Patrick Wendell <pwendell@gmail.com>
Closes#108 from pwendell/ganglia and squashes the following commits:
326712a [Patrick Wendell] Responding to review feedback
5f28ee4 [Patrick Wendell] SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues.
This makes two changes.
1) Spark uses the shaded version of asm that is (conveniently) published
with Kryo.
2) Existing exclude rules around asm are updated to reflect the new groupId
of `org.ow2.asm`. This made all of the old rules not work with newer Hadoop
versions that pull in new asm versions.
Author: Patrick Wendell <pwendell@gmail.com>
Closes#100 from pwendell/asm and squashes the following commits:
9235f3f [Patrick Wendell] SPARK-782 Clean up for ASM dependency.
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes#33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Patrick Wendell <pwendell@gmail.com>
Closes#17 from ScrapCodes/java8-lambdas and squashes the following commits:
95850e6 [Patrick Wendell] Some doc improvements and build changes to the Java 8 patch.
85a954e [Prashant Sharma] Nit. import orderings.
673f7ac [Prashant Sharma] Added support for -java-home as well
80a13e8 [Prashant Sharma] Used fake class tag syntax
26eb3f6 [Prashant Sharma] Patrick's comments on PR.
35d8d79 [Prashant Sharma] Specified java 8 building in the docs
31d4cd6 [Prashant Sharma] Maven build to support -Pjava8-tests flag.
4ab87d3 [Prashant Sharma] Review feedback on the pr
c33dc2c [Prashant Sharma] SPARK-964, Java 8 API Support.
This lets us explicitly include Avro based on a profile for 0.23.X
builds. It makes me sad how convoluted it is to express this logic
in Maven. @tgraves and @sryza curious if this works for you.
I'm also considering just reverting to how it was before. The only
real problem was that Spark advertised a dependency on Avro
even though it only really depends transitively on Avro through
other deps.
Author: Patrick Wendell <pwendell@gmail.com>
Closes#49 from pwendell/avro-build-fix and squashes the following commits:
8d6ee92 [Patrick Wendell] SPARK-1121: Add avro to yarn-alpha profile
(Ported from https://github.com/apache/incubator-spark/pull/650 )
This adds one more change though, to fix the scala version warning introduced by json4s recently.
Author: Sean Owen <sowen@cloudera.com>
Closes#32 from srowen/SPARK-1084.2 and squashes the following commits:
9240abd [Sean Owen] Avoid scala version conflict in scalap induced by json4s dependency
1561cec [Sean Owen] Remove "exclude *" dependencies that are causing Maven warnings, and that are apparently unneeded anyway
This removes some loose ends not caught by the other (incubating -> tlp) patches. @markhamstra this updates the version as you mentioned earlier.
Author: Patrick Wendell <pwendell@gmail.com>
Closes#51 from pwendell/tlp and squashes the following commits:
d553b1b [Patrick Wendell] Remove remaining references to incubation
This update contains a lot of bug fixes and some new perf improvements.
It is also binary compatible with the current 4.0.13.Final
For more information: http://netty.io/news/2014/02/25/4-0-17-Final.html
Author: Binh Nguyen <ngbinh@gmail.com>
Author: Binh Nguyen <ngbinh@gmail.com>
Closes#41 from ngbinh/master and squashes the following commits:
a9498f4 [Binh Nguyen] update io.netty to 4.0.17.Final
(Ported from https://github.com/apache/incubator-spark/pull/637 )
Author: Sean Owen <sowen@cloudera.com>
Closes#31 from srowen/SPARK-1084.1 and squashes the following commits:
6c4a32c [Sean Owen] Suppress warnings about legitimate unchecked array creations, or change code to avoid it
f35b833 [Sean Owen] Fix two misc javadoc problems
254e8ef [Sean Owen] Fix one new style error introduced in scaladoc warning commit
5b2fce2 [Sean Owen] Fix scaladoc invocation warning, and enable javac warnings properly, with plugin config updates
007762b [Sean Owen] Remove dead scaladoc links
b8ff8cb [Sean Owen] Replace deprecated Ant <tasks> with <target>
Author: Prashant Sharma <prashant.s@imaginea.com>
Closes#6 from ScrapCodes/SPARK-1121/avro-dep-fix and squashes the following commits:
9b29e34 [Prashant Sharma] Review feedback on PR
46ed2ad [Prashant Sharma] SPARK-1121-Only add avro if the build is for Hadoop 0.23.X and SPARK_YARN is set
Prompted by a recent thread on the mailing list, I tried and failed to see if Spark can be made independent of log4j. There are a few cases where control of the underlying logging is pretty useful, and to do that, you have to bind to a specific logger.
Instead I propose some tidying that leaves Spark's use of log4j, but gets rid of warnings and should still enable downstream users to switch. The idea is to pipe everything (except log4j) through SLF4J, and have Spark use SLF4J directly when logging, and where Spark needs to output info (REPL and tests), bind from SLF4J to log4j.
This leaves the same behavior in Spark. It means that downstream users who want to use something except log4j should:
- Exclude dependencies on log4j, slf4j-log4j12 from Spark
- Include dependency on log4j-over-slf4j
- Include dependency on another logger X, and another slf4j-X
- Recreate any log config that Spark does, that is needed, in the other logger's config
That sounds about right.
Here are the key changes:
- Include the jcl-over-slf4j shim everywhere by depending on it in core.
- Exclude dependencies on commons-logging from third-party libraries.
- Include the jul-to-slf4j shim everywhere by depending on it in core.
- Exclude slf4j-* dependencies from third-party libraries to prevent collision or warnings
- Added missing slf4j-log4j12 binding to GraphX, Bagel module tests
And minor/incidental changes:
- Update to SLF4J 1.7.5, which happily matches Hadoop 2’s version and is a recommended update over 1.7.2
- (Remove a duplicate HBase dependency declaration in SparkBuild.scala)
- (Remove a duplicate mockito dependency declaration that was causing warnings and bugging me)
Author: Sean Owen <sowen@cloudera.com>
Closes#570 from srowen/SPARK-1071 and squashes the following commits:
52eac9f [Sean Owen] Add slf4j-over-log4j12 dependency to core (non-test) and remove it from things that depend on core.
77a7fa9 [Sean Owen] SPARK-1071: Tidy logging strategy and use of log4j
Version number to 1.0.0-SNAPSHOT
Since 0.9.0-incubating is done and out the door, we shouldn't be building 0.9.0-incubating-SNAPSHOT anymore.
@pwendell
Author: Mark Hamstra <markhamstra@gmail.com>
== Merge branch commits ==
commit 1b00a8a7c1a7f251b4bb3774b84b9e64758eaa71
Author: Mark Hamstra <markhamstra@gmail.com>
Date: Wed Feb 5 09:30:32 2014 -0800
Version number to 1.0.0-SNAPSHOT
Upgrade junit-interface plugin from 0.9 to 0.10.
I noticed that the JavaAPISuite tests didn't
appear to display any output locally or under
Jenkins, making it difficult to know whether they
were running. This change increases the verbosity
to more closely match the ScalaTest tests.
SPARK-998: Support Launching Driver Inside of Standalone Mode
[NOTE: I need to bring the tests up to date with new changes, so for now they will fail]
This patch provides support for launching driver programs inside of a standalone cluster manager. It also supports monitoring and re-launching of driver programs which is useful for long running, recoverable applications such as Spark Streaming jobs. For those jobs, this patch allows a deployment mode which is resilient to the failure of any worker node, failure of a master node (provided a multi-master setup), and even failures of the applicaiton itself, provided they are recoverable on a restart. Driver information, such as the status and logs from a driver, is displayed in the UI
There are a few small TODO's here, but the code is generally feature-complete. They are:
- Bring tests up to date and add test coverage
- Restarting on failure should be optional and maybe off by default.
- See if we can re-use akka connections to facilitate clients behind a firewall
A sensible place to start for review would be to look at the `DriverClient` class which presents users the ability to launch their driver program. I've also added an example program (`DriverSubmissionTest`) that allows you to test this locally and play around with killing workers, etc. Most of the code is devoted to persisting driver state in the cluster manger, exposing it in the UI, and dealing correctly with various types of failures.
Instructions to test locally:
- `sbt/sbt assembly/assembly examples/assembly`
- start a local version of the standalone cluster manager
```
./spark-class org.apache.spark.deploy.client.DriverClient \
-j -Dspark.test.property=something \
-e SPARK_TEST_KEY=SOMEVALUE \
launch spark://10.99.1.14:7077 \
../path-to-examples-assembly-jar \
org.apache.spark.examples.DriverSubmissionTest 1000 some extra options --some-option-here -X 13
```
- Go in the UI and make sure it started correctly, look at the output etc
- Kill workers, the driver program, masters, etc.
Refactored the streaming project to separate external libraries like Twitter, Kafka, Flume, etc.
At a high level, these are the following changes.
1. All the external code was put in `SPARK_HOME/external/` as separate SBT projects and Maven modules. Their artifact names are `spark-streaming-twitter`, `spark-streaming-kafka`, etc. Both SparkBuild.scala and pom.xml files have been updated. References to external libraries and repositories have been removed from the settings of root and streaming projects/modules.
2. To avail the external functionality (say, creating a Twitter stream), the developer has to `import org.apache.spark.streaming.twitter._` . For Scala API, the developer has to call `TwitterUtils.createStream(streamingContext, ...)`. For the Java API, the developer has to call `TwitterUtils.createStream(javaStreamingContext, ...)`.
3. Each external project has its own scala and java unit tests. Note the unit tests of each external library use classes of the streaming unit tests (`TestSuiteBase`, `LocalJavaStreamingContext`, etc.). To enable this code sharing among test classes, `dependsOn(streaming % "compile->compile,test->test")` was used in the SparkBuild.scala . In the streaming/pom.xml, an additional `maven-jar-plugin` was necessary to capture this dependency (see comment inside the pom.xml for more information).
4. Jars of the external projects have been added to examples project but not to the assembly project.
5. In some files, imports have been rearrange to conform to the Spark coding guidelines.
Approximate distinct count
Added countApproxDistinct() to RDD and countApproxDistinctByKey() to PairRDDFunctions to approximately count distinct number of elements and distinct number of values per key, respectively. Both functions use HyperLogLog from stream-lib for counting. Both functions take a parameter that controls the trade-off between accuracy and memory consumption. Also added Scala docs and test suites for both methods.
The correct format is to not have a trailing slash.
For me this caused non-deterministic failures due to issues fetching
certain artifacts. The issue was that some of the maven caches would
fail to fetch the artifact (due to the way that the artifact
path was concatenated with the repository) and this short-circuited
the download process in a silent way. Here is what the log output
looked like:
Downloading: http://repo.maven.apache.org/maven2/org/spark-project/akka/akka-remote_2.10/2.2.3-shaded-protobuf/akka-remote_2.10-2.2.3-shaded-protobuf.pom
[WARNING] The POM for org.spark-project.akka:akka-remote_2.10🫙2.2.3-shaded-protobuf is missing, no dependency information available
This was pretty brutal to debug since there was no error message
anywhere and the path *looks* correct as reported by the Maven log.
Standalone Scheduler fault tolerance using ZooKeeper
This patch implements full distributed fault tolerance for standalone scheduler Masters.
There is only one master Leader at a time, which is actively serving scheduling
requests. If this Leader crashes, another master will eventually be elected, reconstruct
the state from the first Master, and continue serving scheduling requests.
Leader election is performed using the ZooKeeper leader election pattern. We try to minimize
the use of ZooKeeper and the assumptions about ZooKeeper's behavior, so there is a layer of
retries and session monitoring on top of the ZooKeeper client.
Master failover follows directly from the single-node Master recovery via the file
system (patch d5a96fe), save that the Master state is stored in ZooKeeper instead.
Configuration:
By default, no recovery mechanism is enabled (spark.deploy.recoveryMode = NONE).
By setting spark.deploy.recoveryMode to ZOOKEEPER and setting spark.deploy.zookeeper.url
to an appropriate ZooKeeper URL, ZooKeeper recovery mode is enabled.
By setting spark.deploy.recoveryMode to FILESYSTEM and setting spark.deploy.recoveryDirectory
to an appropriate directory accessible by the Master, we will keep the behavior of from d5a96fe.
Additionally, places where a Master could be specificied by a spark:// url can now take
comma-delimited lists to specify backup masters. Note that this is only used for registration
of NEW Workers and application Clients. Once a Worker or Client has registered with the
Master Leader, it is "in the system" and will never need to register again.
Conflicts:
bagel/pom.xml
core/pom.xml
core/src/test/scala/org/apache/spark/ui/UISuite.scala
examples/pom.xml
mllib/pom.xml
pom.xml
project/SparkBuild.scala
repl/pom.xml
streaming/pom.xml
tools/pom.xml
In scala 2.10, a shorter representation is used for naming artifacts
so changed to shorter scala version for artifacts and made it a property in pom.
This patch implements full distributed fault tolerance for standalone scheduler Masters.
There is only one master Leader at a time, which is actively serving scheduling
requests. If this Leader crashes, another master will eventually be elected, reconstruct
the state from the first Master, and continue serving scheduling requests.
Leader election is performed using the ZooKeeper leader election pattern. We try to minimize
the use of ZooKeeper and the assumptions about ZooKeeper's behavior, so there is a layer of
retries and session monitoring on top of the ZooKeeper client.
Master failover follows directly from the single-node Master recovery via the file
system (patch 194ba4b8), save that the Master state is stored in ZooKeeper instead.
Configuration:
By default, no recovery mechanism is enabled (spark.deploy.recoveryMode = NONE).
By setting spark.deploy.recoveryMode to ZOOKEEPER and setting spark.deploy.zookeeper.url
to an appropriate ZooKeeper URL, ZooKeeper recovery mode is enabled.
By setting spark.deploy.recoveryMode to FILESYSTEM and setting spark.deploy.recoveryDirectory
to an appropriate directory accessible by the Master, we will keep the behavior of from 194ba4b8.
Additionally, places where a Master could be specificied by a spark:// url can now take
comma-delimited lists to specify backup masters. Note that this is only used for registration
of NEW Workers and application Clients. Once a Worker or Client has registered with the
Master Leader, it is "in the system" and will never need to register again.
Forthcoming:
Documentation, tests (! - only ad hoc testing has been performed so far)
I do not intend for this commit to be merged until tests are added, but this patch should
still be mostly reviewable until then.
This includes the following changes:
- The "assembly" package now builds in Maven by default, and creates an
assembly containing both hadoop-client and Spark, unlike the old
BigTop distribution assembly that skipped hadoop-client
- There is now a bigtop-dist package to build the old BigTop assembly
- The repl-bin package is no longer built by default since the scripts
don't reply on it; instead it can be enabled with -Prepl-bin
- Py4J is now included in the assembly/lib folder as a local Maven repo,
so that the Maven package can link to it
- run-example now adds the original Spark classpath as well because the
Maven examples assembly lists spark-core and such as provided
- The various Maven projects add a spark-yarn dependency correctly
It made the JSON creation slightly more complicated, but reduces one external dependency. The scala library also properly escape "/" (which lift-json doesn't).
This is used to find methods in the Scala API that
need to be ported to the Java API. To use it:
./run spark.tools.JavaAPICompletenessChecker
Conflicts:
project/SparkBuild.scala
run
run2.cmd