aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* Remove blank lines added to Spark coreAnkur Dave2014-01-102-8/+0
|
* Revert changes to Spark's (PrimitiveKey)OpenHashMap; copy PKOHM to graphxAnkur Dave2014-01-102-75/+33
|
* Removed Kryo dependency and graphx-shellAnkur Dave2014-01-092-2/+2
|
* Fix mis-merge in 44fd30d3fbcf830deecbe8ea3e8ea165e74e6eddAnkur Dave2014-01-081-0/+5
|
* Merge remote-tracking branch 'spark-upstream/master' into HEADAnkur Dave2014-01-08217-3800/+5624
|\ | | | | | | | | | | | | | | | | | | | | Conflicts: README.md core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala pom.xml project/SparkBuild.scala repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
| * Merge pull request #357 from hsaputra/set_boolean_paramnameReynold Xin2014-01-082-3/+4
| |\ | | | | | | | | | | | | | | | Set boolean param name for call to SparkHadoopMapReduceUtil.newTaskAttemptID Set boolean param name for call to SparkHadoopMapReduceUtil.newTaskAttemptID to make it clear which param being set.
| | * Resolve PR review over 100 charsHenry Saputra2014-01-081-1/+2
| | |
| | * Set boolean param name for two files call to ↵Henry Saputra2014-01-072-3/+3
| | | | | | | | | | | | | | | | | | SparkHadoopMapReduceUtil.newTaskAttemptID to make it clear which param being set.
| * | Merge pull request #356 from hsaputra/remove_deprecated_cleanup_methodReynold Xin2014-01-082-6/+0
| |\ \ | | |/ | |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Remove calls to deprecated mapred's OutputCommitter.cleanupJob Since Hadoop 1.0.4 the mapred OutputCommitter.commitJob should do cleanup job via call to OutputCommitter.cleanupJob, Remove SparkHadoopWriter.cleanup since it is used only by PairRDDFunctions. In fact the implementation of mapred OutputCommitter.commitJob looks like this: public void commitJob(JobContext jobContext) throws IOException { cleanupJob(jobContext); }
| | * Remove calls to deprecated mapred's OutputCommitter.cleanupJob because since ↵Henry Saputra2014-01-072-6/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Hadoop 1.0.4 the mapred OutputCommitter.commitJob should do cleanup job. In fact the implementation of mapred OutputCommitter.commitJob looks like this: public void commitJob(JobContext jobContext) throws IOException { cleanupJob(jobContext); } (The jobContext input argument is type of org.apache.hadoop.mapred.JobContext)
| * | Merge pull request #336 from liancheng/akka-remote-lookupPatrick Wendell2014-01-078-38/+30
| |\ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | Get rid of `Either[ActorRef, ActorSelection]' In this pull request, instead of returning an `Either[ActorRef, ActorSelection]`, `registerOrLookup` identifies the remote actor blockingly to obtain an `ActorRef`, or throws an exception if the remote actor doesn't exist or the lookup times out (configured by `spark.akka.lookupTimeout`). This function is only called when an `SparkEnv` is constructed (instantiating driver or executor), so the blocking call is considered acceptable. Executor side `ActorSelection`s/`ActorRef`s to driver side `MapOutputTrackerMasterActor` and `BlockManagerMasterActor` are affected by this pull request. `ActorSelection` is dangerous and should be used with care. It's only absolutely safe to send messages via an `ActorSelection` when the remote actor is stateless, so that actor incarnation is irrelevant. But as pointed by @ScrapCodes in the comments below, executor exits immediately once the connection to the driver lost, `ActorSelection`s are not harmful in this scenario. So this pull request is mostly a code style patch.
| | * | Fixed test suite compilation errorsLian, Cheng2014-01-061-3/+3
| | | |
| | * | Fixed several compilation errors in test suitesLian, Cheng2014-01-062-5/+8
| | | |
| | * | Get rid of `Either[ActorRef, ActorSelection]'Lian, Cheng2014-01-066-30/+19
| | | | | | | | | | | | | | | | Although we can send messages via an ActorSelection, it would be better to identify the actor and obtain an ActorRef first, so that we can get informed earlier if the remote actor doesn't exist, and get rid of the annoying Either wrapper.
| * | | Merge pull request #350 from mateiz/standalone-limitMatei Zaharia2014-01-089-12/+28
| |\ \ \ | | |_|/ | |/| | | | | | | | | | | | | | Add way to limit default # of cores used by apps in standalone mode Also documents the spark.deploy.spreadOut option, and fixes a config option that had a dash in its name.
| | * | Address review commentsMatei Zaharia2014-01-077-6/+9
| | | |
| | * | Fix unit test compilationMatei Zaharia2014-01-071-1/+2
| | | |
| | * | Add way to limit default # of cores used by applications on standalone modeMatei Zaharia2014-01-074-6/+18
| | | | | | | | | | | | | | | | Also documents the spark.deploy.spreadOut option.
| * | | Merge pull request #352 from markhamstra/oldArchPatrick Wendell2014-01-071-8/+2
| |\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | Don't leave os.arch unset after BlockManagerSuite Recent SparkConf changes meant that BlockManagerSuite was now leaving the os.arch System.property unset. That's a problem for any subsequent tests that rely upon having a valid os.arch. This is true for CompressionCodecSuite in the usual maven build test order, even though it isn't usually true for the sbt build.
| | * | | Fix BlockManagerSuite#afterMark Hamstra2014-01-071-8/+2
| | |/ /
| * / / Add log4j exclusion rule to maven.Patrick Wendell2014-01-072-4/+6
| |/ / | | | | | | | | | | | | | | | | | | | | | | | | | | | To make this work I had to rename the defaults file. Otherwise maven's pattern matching rules included it when trying to match other log4j.properties files. I also fixed a bug in the existing maven build where two <transformers> tags were present in assembly/pom.xml such that one overwrote the other.
| * | Merge pull request #318 from srowen/masterReynold Xin2014-01-076-38/+39
| |\ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance Sorry if this is too abrupt or not a welcome set of changes, but thought I'd see if I could contribute a little. I'm a Java developer and just getting seriously into Spark. So I thought I'd suggest a number of small changes to the couple Java parts of the code to make it a little tighter, more standard and even a bit faster. Feel free to take all, some or none of this. Happy to explain any of it.
| | * \ Merge remote-tracking branch 'upstream/master'Sean Owen2014-01-0621-248/+520
| | |\ \
| | * | | Suggested small changes to Java code for slightly more standard style, ↵Sean Owen2014-01-026-38/+39
| | | | | | | | | | | | | | | | | | | | encapsulation and in some cases performance
| * | | | formatting related fixes suggested by Patrick.Prashant Sharma2014-01-071-5/+1
| | | | |
| * | | | Allow configuration to be printed in logs for diagnosis.Prashant Sharma2014-01-071-0/+4
| | | | |
| * | | | Allow users to set arbitrary akka configurations via spark conf.Prashant Sharma2014-01-072-3/+12
| | | | |
| * | | | Fix test breaking downstream buildsPatrick Wendell2014-01-061-1/+1
| | | | |
| * | | | Merge pull request #330 from tgravescs/fix_addjars_null_handlingPatrick Wendell2014-01-061-2/+3
| |\ \ \ \ | | |_|/ / | |/| | | | | | | | | | | | | | | | | | Fix handling of empty SPARK_EXAMPLES_JAR Currently if SPARK_EXAMPLES_JAR is left unset you get a null pointer exception when running the examples (atleast on spark on yarn). The null now gets turned into a string of "null" when its put into the SparkConf so addJar no longer properly ignores it. This fixes that so that it can be left unset.
| | * | | Add warning to null setJars checkThomas Graves2014-01-061-1/+2
| | | | |
| | * | | Fix handling of empty SPARK_EXAMPLES_JARThomas Graves2014-01-041-1/+1
| | | |/ | | |/|
| * | | Merge pull request #333 from pwendell/logging-silencePatrick Wendell2014-01-052-3/+25
| |\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Quiet ERROR-level Akka Logs This fixes an issue I've seen where akka logs a bunch of things at ERROR level when connecting to a standalone cluster, even in the normal case. I noticed that even when lifecycle logging was disabled, the netty code inside of akka still logged away via akka's EndpointWriter class. There are also some other log streams that I think are new in akka 2.2.1 that I've disabled. Finally, I added some better logging to the standalone client. This makes it more clear when a connection failure occurs what is going on. Previously it never explicitly said if a connection attempt had failed. The commit messages here have some more detail.
| | * | | Responding to Aaron's reviewPatrick Wendell2014-01-051-0/+2
| | | | |
| | * | | Provide logging when attempts to connect to the master fail.Patrick Wendell2014-01-051-1/+11
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Without these it's a bit less clear what's going on for the user. One thing I realize when doing this is that akka itself actually retries the initial association. So the retry we currently have is redundant with akka's.
| | * | | Quite akka when remote lifecycle logging is disabled.Patrick Wendell2014-01-051-2/+12
| | |/ / | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | I noticed when connecting to a standalone cluster Spark gives a bunch of Akka ERROR logs that make it seem like something is failing. This patch does two things: 1. Akka dead letter logging is turned on/off according to the existing lifecycle spark property. 2. We explicitly silence akka's EndpointWriter log in log4j. This is necessary because for some reason that log doesn't pick up on the lifecycle logging settings. After a few hours of debugging this was the only solution I found that worked.
| * | | Merge pull request #334 from pwendell/examples-fixReynold Xin2014-01-051-0/+6
| |\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Removing SPARK_EXAMPLES_JAR in the code This re-writes all of the examples to use the `SparkContext.jarOfClass` mechanism for loading the examples jar. This necessary for environments like YARN and the Standalone mode where example programs will be submit from inside the cluster rather than at the client using `./spark-example`. This still leaves SPARK_EXAMPLES_JAR in place in the shell scripts for setting up the classpath if `./spark-example` is run.
| | * | | Removing SPARK_EXAMPLES_JAR in the codePatrick Wendell2014-01-051-0/+6
| | |/ /
| * / / Fall back to zero-arg constructor for Serializer initialization if there is ↵Reynold Xin2014-01-052-2/+16
| |/ / | | | | | | | | | | | | | | | no constructor that accepts SparkConf. This maintains backward compatibility with older serializers implemented by users.
| * | Merge remote-tracking branch 'apache-github/master' into remove-binariesPatrick Wendell2014-01-034-7/+7
| |\ \ | | | | | | | | | | | | | | | | | | | | Conflicts: core/src/test/scala/org/apache/spark/DriverSuite.scala docs/python-programming-guide.md
| | * \ Merge pull request #317 from ScrapCodes/spark-915-segregate-scriptsPatrick Wendell2014-01-034-7/+7
| | |\ \ | | | | | | | | | | | | | | | Spark-915 segregate scripts
| | | * | sbin/compute-classpath* bin/compute-classpath*Prashant Sharma2014-01-031-1/+1
| | | | |
| | | * | sbin/spark-class* -> bin/spark-class*Prashant Sharma2014-01-033-5/+5
| | | | |
| | | * | Merge branch 'scripts-reorg' of github.com:shane-huang/incubator-spark into ↵Prashant Sharma2014-01-025-7/+7
| | | |\ \ | | | | |/ | | | |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | spark-915-segregate-scripts Conflicts: bin/spark-shell core/pom.xml core/src/main/scala/org/apache/spark/SparkContext.scala core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala core/src/test/scala/org/apache/spark/DriverSuite.scala python/run-tests sbin/compute-classpath.sh sbin/spark-class sbin/stop-slaves.sh
| | | | * deprecate "spark" script and SPAKR_CLASSPATH environment variableAndrew xia2013-10-122-2/+1
| | | | |
| | | | * Merge branch 'reorgscripts' into scripts-reorgshane-huang2013-09-275-7/+7
| | | | |\
| | | | | * fix paths and change spark to use APP_MEM as application driver memory ↵shane-huang2013-09-261-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | instead of SPARK_MEM, user should add application jars to SPARK_CLASSPATH Signed-off-by: shane-huang <shengsheng.huang@intel.com>
| | | | | * fix pathshane-huang2013-09-261-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | Signed-off-by: shane-huang <shengsheng.huang@intel.com>
| | | | | * added spark-class and spark-executor to sbinshane-huang2013-09-234-6/+6
| | | | | | | | | | | | | | | | | | | | | | | | Signed-off-by: shane-huang <shengsheng.huang@intel.com>
| * | | | | Changes on top of Prashant's patch.Patrick Wendell2014-01-034-54/+35
| | | | | | | | | | | | | | | | | | | | | | | | Closes #316
| * | | | | Restored the previously removed testPrashant Sharma2014-01-031-1/+12
| | | | | |