Commit graph

317 commits

Author SHA1 Message Date
Reynold Xin 6bcac986b2 Merge branch 'master' of github.com:apache/incubator-spark 2013-11-25 15:47:47 +08:00
Matei Zaharia 859d62dc2a Merge pull request #151 from russellcardullo/add-graphite-sink
Add graphite sink for metrics

This adds a metrics sink for graphite.  The sink must
be configured with the host and port of a graphite node
and optionally may be configured with a prefix that will
be prepended to all metrics that are sent to graphite.
2013-11-24 16:19:51 -08:00
Matei Zaharia 9290e5bcd2 Merge pull request #165 from NathanHowell/kerberos-master
spark-assembly.jar fails to authenticate with YARN ResourceManager

The META-INF/services/ sbt MergeStrategy was discarding support for Kerberos, among others. This pull request changes to a merge strategy similar to sbt-assembly's default. I've also included an update to sbt-assembly 0.9.2, a minor fix to it's zip file handling.
2013-11-13 16:48:44 -08:00
Matei Zaharia f49ea28d25 Merge pull request #137 from tgravescs/sparkYarnJarsHdfsRebase
Allow spark on yarn to be run from HDFS.

Allows the spark.jar, app.jar, and log4j.properties to be put into hdfs.  Allows you to specify the files on a different hdfs cluster and it will copy them over. It makes sure permissions are correct and makes sure to put things into public distributed cache so they can be reused amongst users if their permissions are appropriate.  Also add a bit of error handling for missing arguments.
2013-11-12 19:13:39 -08:00
Nathan Howell 48eac0bcbf Upgrade to sbt-assembly 0.9.2 2013-11-12 13:29:25 -08:00
Nathan Howell 23146a6705 spark-assembly.jar fails to authenticate with YARN ResourceManager
sbt-assembly is setup to pick the first META-INF/services/org.apache.hadoop.security.SecurityInfo file instead of merging them. This causes Kerberos authentication to fail, this manifests itself in the "info:null" debug log statement:

    DEBUG SaslRpcClient: Get token info proto:interface org.apache.hadoop.yarn.api.ApplicationClientProtocolPB info:null
    DEBUG SaslRpcClient: Get kerberos info proto:interface org.apache.hadoop.yarn.api.ApplicationClientProtocolPB info:null
    ERROR UserGroupInformation: PriviledgedActionException as:foo@BAR (auth:KERBEROS) cause:org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
    DEBUG UserGroupInformation: PrivilegedAction as:foo@BAR (auth:KERBEROS) from:org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:583)
    WARN Client: Exception encountered while connecting to the server : org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
    ERROR UserGroupInformation: PriviledgedActionException as:foo@BAR (auth:KERBEROS) cause:java.io.IOException: org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]

This previously would just contain a single class:

$ unzip -c assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar META-INF/services/org.apache.hadoop.security.SecurityInfo
Archive:  assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar
  inflating: META-INF/services/org.apache.hadoop.security.SecurityInfo

    org.apache.hadoop.security.AnnotatedSecurityInfo

And now has the full list of classes:

$ unzip -c assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar META-INF/services/org.apache.hadoop.security.SecurityInfoArchive:  assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar
  inflating: META-INF/services/org.apache.hadoop.security.SecurityInfo

    org.apache.hadoop.security.AnnotatedSecurityInfo
    org.apache.hadoop.mapreduce.v2.app.MRClientSecurityInfo
    org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo
    org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo
    org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo
    org.apache.hadoop.yarn.security.SchedulerSecurityInfo
    org.apache.hadoop.yarn.security.admin.AdminSecurityInfo
    org.apache.hadoop.yarn.server.RMNMSecurityInfoClass
2013-11-12 13:27:50 -08:00
tgravescs 17bb9a27b2 Add mockito to the sbt build 2013-11-11 10:01:23 -06:00
Josh Rosen a37ff0f1db Add spark-tools assembly to spark-class classpath.
This allows the JavaAPICompletenessChecker to be
run with Spark 0.8+.
2013-11-09 13:42:45 -08:00
Russell Cardullo ef85a51f85 Add graphite sink for metrics
This adds a metrics sink for graphite.  The sink must
be configured with the host and port of a graphite node
and optionally may be configured with a prefix that will
be prepended to all metrics that are sent to graphite.
2013-11-08 16:36:03 -08:00
Ankur Dave 5064f9b2d2 Merge remote-tracking branch 'spark-upstream/master'
Conflicts:
	project/SparkBuild.scala
2013-10-30 15:59:09 -07:00
Patrick Wendell af4a529f6e Exclude jopt from kafka dependency.
Kafka uses an older version of jopt that causes bad conflicts with the version
used by spark-perf. It's not easy to remove this downstream because of the way
that spark-perf uses Spark (by including a spark assembly as an unmanaged jar).
This fixes the problem at its source by just never including it.
2013-10-25 09:20:30 -07:00
Matei Zaharia dadfc63b03 Fix Maven build to use MQTT repository 2013-10-23 15:29:22 -07:00
Matei Zaharia dd659642e7 Merge pull request #64 from prabeesh/master
MQTT Adapter for Spark Streaming

MQTT is a machine-to-machine (M2M)/Internet of Things connectivity protocol.
It was designed as an extremely lightweight publish/subscribe messaging transport. You may read more about it here http://mqtt.org/

Message Queue Telemetry Transport (MQTT) is an open message protocol for M2M communications. It enables the transfer of telemetry-style data in the form of messages from devices like sensors and actuators, to mobile phones, embedded systems on vehicles, or laptops and full scale computers.

The protocol was invented by Andy Stanford-Clark of IBM, and Arlen Nipper of Cirrus Link Solutions

This protocol enables a publish/subscribe messaging model in an extremely lightweight way. It is useful for connections with remote locations where line of code and network bandwidth is a constraint.

MQTT is one of the widely used protocol for 'Internet of Things'. This protocol is getting much attraction as anything and everything is getting connected to internet and they all produce data. Researchers and companies predict some 25 billion devices will be connected to the internet by 2015.

Plugin/Support for MQTT is available in popular MQs like RabbitMQ, ActiveMQ etc.

Support for MQTT in Spark will help people with Internet of Things (IoT) projects to use Spark Streaming for their real time data processing needs (from sensors and other embedded devices etc).
2013-10-23 15:07:59 -07:00
Matei Zaharia 731c94e91d Merge pull request #56 from jerryshao/kafka-0.8-dev
Upgrade Kafka 0.7.2 to Kafka 0.8.0-beta1 for Spark Streaming

Conflicts:
	streaming/pom.xml
2013-10-21 23:31:38 -07:00
Matei Zaharia 8de9706b86 Merge pull request #66 from shivaram/sbt-assembly-deps
Add SBT target to assemble dependencies

This pull request is an attempt to address the long assembly build times during development. Instead of rebuilding the assembly jar for every Spark change, this pull request adds a new SBT target `spark` that packages all the Spark modules and builds an assembly of the dependencies.

So the work flow that should work now would be something like

```
./sbt/sbt spark # Doing this once should suffice
## Make changes
./sbt/sbt compile
./sbt/sbt test or ./spark-shell
```
2013-10-18 20:32:39 -07:00
Joseph E. Gonzalez 1856b37e9d Merge branch 'master' of https://github.com/apache/incubator-spark into indexedrdd_graphx 2013-10-18 12:21:19 -07:00
prabeesh 29245605bf remove unused dependency 2013-10-17 09:57:30 +05:30
Shivaram Venkataraman 0a4b76fcc2 Rename SBT target to assemble-deps. 2013-10-16 17:05:46 -07:00
prabeesh 06de3d516d added mqtt adapter library dependencies 2013-10-16 13:38:37 +05:30
Patrick Wendell 35befe07bb Fixing spark streaming example and a bug in examples build.
- Examples assembly included a log4j.properties which clobbered Spark's
- Example had an error where some classes weren't serializable
- Did some other clean-up in this example
2013-10-15 22:55:43 -07:00
Shivaram Venkataraman 051cd960d9 Merge branch 'master' of https://github.com/apache/incubator-spark into sbt-assembly-deps 2013-10-15 13:26:40 -07:00
Joseph E. Gonzalez ef7c369092 merged with upstream changes 2013-10-14 22:56:42 -07:00
jerryshao c23cd72b4b Upgrade Kafka 0.7.2 to Kafka 0.8.0-beta1 for Spark Streaming 2013-10-12 20:00:42 +08:00
Shivaram Venkataraman c441904bce Add a comment and exclude tools 2013-10-11 18:23:15 -07:00
Matei Zaharia c71499b779 Merge pull request #19 from aarondav/master-zk
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.
2013-10-10 17:16:42 -07:00
Shivaram Venkataraman 484166d520 Add new SBT target for dependency assembly 2013-10-09 04:24:34 -07:00
Reynold Xin 213b70a2db Merge pull request #31 from sundeepn/branch-0.8
Resolving package conflicts with hadoop 0.23.9

Hadoop 0.23.9 is having a package conflict with easymock's dependencies.

(cherry picked from commit 023e3fdf00)
Signed-off-by: Reynold Xin <rxin@apache.org>
2013-10-07 10:54:22 -07:00
Du Li 9fd6bba60d ask ivy/sbt to check local maven repo under ~/.m2 2013-10-01 15:46:51 -07:00
Aaron Davidson f549ea33d3 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 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.
2013-09-26 15:04:23 -07:00
Reynold Xin 3f283278b0 Removed scala -optimize flag. 2013-09-26 13:58:10 -07:00
Reynold Xin c514cd1587 Merge pull request #930 from holdenk/master
Add mapPartitionsWithIndex
2013-09-26 13:48:20 -07:00
Patrick Wendell 6079721fa1 Update build version in master 2013-09-24 11:41:51 -07:00
Joseph E. Gonzalez 55696e2584 GraphX now builds with all merged changes. 2013-09-17 22:42:12 -07:00
Joseph E. Gonzalez 8b59fb72c4 Merging latest changes from spark main branch 2013-09-17 20:56:12 -07:00
Patrick Wendell c856860c5b Bumping Mesos version to 0.13.0 2013-09-15 12:46:26 -07:00
Holden Karau 68068977b8 Fix build on ubuntu 2013-09-14 20:51:11 -07:00
Patrick Wendell 91a59e6b10 Merge pull request #919 from mateiz/jets3t
Add explicit jets3t dependency, which is excluded in hadoop-client
2013-09-11 10:21:48 -07:00
Patrick Wendell 0c1985b153 Fix HDFS access bug with assembly build.
Due to this change in HDFS:
https://issues.apache.org/jira/browse/HADOOP-7549

there is a bug when using the new assembly builds. The symptom is that any HDFS access
results in an exception saying "No filesystem for scheme 'hdfs'". This adds a merge
strategy in the assembly build which fixes the problem.
2013-09-10 22:05:13 -07:00
Matei Zaharia f117dc6d0d Add explicit jets3t dependency, which is excluded in hadoop-client 2013-09-10 06:39:25 +00:00
Patrick Wendell f68848d95d Merge pull request #906 from pwendell/ganglia-sink
Clean-up of Metrics Code/Docs and Add Ganglia Sink
2013-09-08 18:32:16 -07:00
Matei Zaharia 0b957997ad Merge pull request #908 from pwendell/master
Fix target JVM version in scala build
2013-09-08 15:30:16 -07:00
Patrick Wendell 27bd74c8ad Fix target JVM version in scala build 2013-09-08 14:37:45 -07:00
Patrick Wendell 8de8ee5d3c Ganglia sink 2013-09-08 10:08:18 -07:00
Patrick Wendell a8e376ec0f Merge pull request #904 from pwendell/master
Adding Apache license to two files
2013-09-07 21:16:01 -07:00
Patrick Wendell 6d2198643c Adding Apache license to two files 2013-09-07 20:46:58 -07:00
Jey Kottalam 30a32c8335 Minor YARN build cleanups 2013-09-06 11:31:16 -07:00
Matei Zaharia 59218bdd49 Add Apache parent POM 2013-09-02 18:34:03 -07:00
Matei Zaharia 5701eb92c7 Fix some URLs 2013-09-01 14:13:16 -07:00
Matei Zaharia 46eecd110a Initial work to rename package to org.apache.spark 2013-09-01 14:13:13 -07:00
Matei Zaharia 666d93c294 Update Maven build to create assemblies expected by new scripts
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
2013-08-29 21:19:06 -07:00