aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* Remove the semicolons at the end of Scala code to make it more pure Scala code.Henry Saputra2013-11-1918-62/+56
| | | | | | | Also remove unused imports as I found them along the way. Remove return statements when returning value in the Scala code. Passing compile and tests.
* Merge pull request #182 from rxin/vectorReynold Xin2013-11-172-15/+148
|\ | | | | | | | | | | | | | | | | Slightly enhanced PrimitiveVector: 1. Added trim() method 2. Added size method. 3. Renamed getUnderlyingArray to array. 4. Minor documentation update.
| * Merge pull request #3 from aarondav/pv-testReynold Xin2013-11-172-0/+120
| |\ | | | | | | Add PrimitiveVectorSuite and fix bug in resize()
| | * Add PrimitiveVectorSuite and fix bug in resize()Aaron Davidson2013-11-172-0/+120
| |/
| * Return the vector itself for trim and resize method in PrimitiveVector.Reynold Xin2013-11-171-3/+4
| |
| * Slightly enhanced PrimitiveVector:Reynold Xin2013-11-171-14/+26
|/ | | | | | | 1. Added trim() method 2. Added size method. 3. Renamed getUnderlyingArray to array. 4. Minor documentation update.
* Merge pull request #178 from hsaputra/simplecleanupcodeMatei Zaharia2013-11-168-15/+10
|\ | | | | | | | | | | | | | | | | Simple cleanup on Spark's Scala code Simple cleanup on Spark's Scala code while testing some modules: -) Remove some of unused imports as I found them -) Remove ";" in the imports statements -) Remove () at the end of method calls like size that does not have size effect.
| * Simple cleanup on Spark's Scala code while testing core and yarn modules:Henry Saputra2013-11-158-15/+10
| | | | | | | | | | | | -) Remove some of unused imports as I found them -) Remove ";" in the imports statements -) Remove () at the end of method call like size that does not have size effect.
* | Merge pull request #173 from kayousterhout/scheduler_hangMatei Zaharia2013-11-141-10/+3
|\ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Fix bug where scheduler could hang after task failure. When a task fails, we need to call reviveOffers() so that the task can be rescheduled on a different machine. In the current code, the state in ClusterTaskSetManager indicating which tasks are pending may be updated after revive offers is called (there's a race condition here), so when revive offers is called, the task set manager does not yet realize that there are failed tasks that need to be relaunched. This isn't currently unit tested but will be once my pull request for merging the cluster and local schedulers goes in -- at which point many more of the unit tests will exercise the code paths through the cluster scheduler (currently the failure test suite uses the local scheduler, which is why we didn't see this bug before).
| * | Fix bug where scheduler could hang after task failure.Kay Ousterhout2013-11-141-10/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | When a task fails, we need to call reviveOffers() so that the task can be rescheduled on a different machine. In the current code, the state in ClusterTaskSetManager indicating which tasks are pending may be updated after revive offers is called (there's a race condition here), so when revive offers is called, the task set manager does not yet realize that there are failed tasks that need to be relaunched.
* | | Merge pull request #175 from kayousterhout/no_retry_not_serializableMatei Zaharia2013-11-141-0/+9
|\ \ \ | | | | | | | | | | | | | | | | | | | | Don't retry tasks when they fail due to a NotSerializableException As with my previous pull request, this will be unit tested once the Cluster and Local schedulers get merged.
| * | | Don't retry tasks when they fail due to a NotSerializableExceptionKay Ousterhout2013-11-141-0/+9
| |/ /
* | | Merge pull request #174 from ahirreddy/masterMatei Zaharia2013-11-141-11/+1
|\ \ \ | |/ / |/| | | | | | | | | | | Write Spark UI url to driver file on HDFS This makes the SIMR code path simpler
| * | Write Spark UI url to driver file on HDFSAhir Reddy2013-11-131-11/+1
| |/
* | Merge pull request #169 from kayousterhout/mesos_fixReynold Xin2013-11-141-0/+1
|\ \ | | | | | | | | | | | | | | | Don't ignore spark.cores.max when using Mesos Coarse mode totalCoresAcquired is decremented but never incremented, causing Spark to effectively ignore spark.cores.max in coarse grained Mesos mode.
| * | Don't ignore spark.cores.max when using Mesos Coarse modeKay Ousterhout2013-11-131-0/+1
| | |
* | | Merge pull request #170 from liancheng/hadooprdd-doc-typoReynold Xin2013-11-141-1/+1
|\ \ \ | | | | | | | | | | | | Fixed a scaladoc typo in HadoopRDD.scala
| * | | Fixed a scaladoc typo in HadoopRDD.scalaLian, Cheng2013-11-141-1/+1
| |/ /
* | | Merge pull request #171 from RIA-pierre-borckmans/masterReynold Xin2013-11-141-2/+2
|\ \ \ | |/ / |/| | | | | | | | | | | Fixed typos in the CDH4 distributions version codes. Nothing important, but annoying when doing a copy/paste...
| * | Fixed typos in the CDH4 distributions version codes.RIA-pierre-borckmans2013-11-141-2/+2
|/ /
* | Merge pull request #159 from liancheng/dagscheduler-actor-refineMatei Zaharia2013-11-134-68/+43
|\ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Migrate the daemon thread started by DAGScheduler to Akka actor `DAGScheduler` adopts an event queue and a daemon thread polling the it to process events sent to a `DAGScheduler`. This is a classical actor use case. By migrating this thread to Akka actor, we may benefit from both cleaner code and better performance (context switching cost of Akka actor is much less than that of a native thread). But things become a little complicated when taking existing test code into consideration. Code in `DAGSchedulerSuite` is somewhat tightly coupled with `DAGScheduler`, and directly calls `DAGScheduler.processEvent` instead of posting event messages to `DAGScheduler`. To minimize code change, I chose to let the actor to delegate messages to `processEvent`. Maybe this doesn't follow conventional actor usage, but I tried to make it apparently correct. Another tricky part is that, since `DAGScheduler` depends on the `ActorSystem` provided by its field `env`, `env` cannot be null. But the `dagScheduler` field created in `DAGSchedulerSuite.before` was given a null `env`. What's more, `BlockManager.blockIdsToBlockManagers` checks whether `env` is null to determine whether to run the production code or the test code (bad smell here, huh?). I went through all callers of `BlockManager.blockIdsToBlockManagers`, and made sure that if `env != null` holds, then `blockManagerMaster == null` must also hold. That's the logic behind `BlockManager.scala` [line 896](https://github.com/liancheng/incubator-spark/compare/dagscheduler-actor-refine?expand=1#diff-2b643ea78c1add0381754b1f47eec132L896). At last, since `DAGScheduler` instances are always `start()`ed after creation, I removed the `start()` method, and starts the `eventProcessActor` within the constructor.
| * | Made some changes according to suggestions from @aarondavLian, Cheng2013-11-112-5/+6
| | |
| * | Put the periodical resubmitFailedStages() call into a scheduled taskLian, Cheng2013-11-111-16/+12
| | |
| * | Remove unnecessary null checkingLian, Cheng2013-11-091-3/+1
| | |
| * | Replaced the daemon thread started by DAGScheduler with an actorLian, Cheng2013-11-094-65/+45
| | |
* | | Merge pull request #165 from NathanHowell/kerberos-masterMatei Zaharia2013-11-132-2/+2
|\ \ \ | |_|/ |/| | | | | | | | | | | 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.
| * | Upgrade to sbt-assembly 0.9.2Nathan Howell2013-11-121-1/+1
| | |
| * | spark-assembly.jar fails to authenticate with YARN ResourceManagerNathan Howell2013-11-121-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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
* | | Merge pull request #166 from ahirreddy/simr-spark-uiMatei Zaharia2013-11-131-0/+12
|\ \ \ | | | | | | | | | | | | | | | | | | | | SIMR Backend Scheduler will now write Spark UI URL to HDFS, which is to ... ...be retrieved by SIMR clients
| * | | SIMR Backend Scheduler will now write Spark UI URL to HDFS, which is to be ↵Ahir Reddy2013-11-121-0/+12
| |/ / | | | | | | | | | retrieved by SIMR clients
* | | Merge pull request #137 from tgravescs/sparkYarnJarsHdfsRebaseMatei Zaharia2013-11-129-173/+655
|\ \ \ | | | | | | | | | | | | | | | | | | | | 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.
| * | | Add mockito to the sbt buildtgravescs2013-11-111-1/+2
| | | |
| * | | Allow spark on yarn to be run from HDFS. Allows the spark.jar, app.jar, and ↵tgravescs2013-11-048-172/+653
| | | | | | | | | | | | | | | | log4j.properties to be put into hdfs.
* | | | Merge pull request #153 from ankurdave/stop-spot-clusterMatei Zaharia2013-11-121-19/+32
|\ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Enable stopping and starting a spot cluster Clusters launched using `--spot-price` contain an on-demand master and spot slaves. Because EC2 does not support stopping spot instances, the spark-ec2 script previously could only destroy such clusters. This pull request makes it possible to stop and restart a spot cluster. * The `stop` command works as expected for a spot cluster: the master is stopped and the slaves are terminated. * To start a stopped spot cluster, the user must invoke `launch --use-existing-master`. This launches fresh spot slaves but resumes the existing master.
| * | | | Enable stopping and starting a spot clusterAnkur Dave2013-11-111-19/+32
| | | | |
* | | | | Merge pull request #160 from xiajunluan/JIRA-923Matei Zaharia2013-11-122-10/+32
|\ \ \ \ \ | |_|_|/ / |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Fix bug JIRA-923 Fix column sort issue in UI for JIRA-923. https://spark-project.atlassian.net/browse/SPARK-923 Conflicts: core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
| * | | | fix format errorAndrew xia2013-11-111-6/+6
| | | | |
| * | | | cut lines to less than 100Andrew xia2013-11-111-2/+4
| | | | |
| * | | | Fix bug JIRA-923Andrew xia2013-11-113-9/+28
| | | | |
* | | | | Merge pull request #164 from tdas/kafka-fixMatei Zaharia2013-11-122-7/+80
|\ \ \ \ \ | |_|/ / / |/| | | | | | | | | | | | | | | | | | | Made block generator thread safe to fix Kafka bug. This is a very important bug fix. Data can and was being lost in the kafka due to this.
| * | | | Made block generator thread safe to fix Kafka bug.Tathagata Das2013-11-122-7/+80
|/ / / /
* | | | Merge pull request #156 from haoyuan/masterMatei Zaharia2013-11-111-7/+8
|\ \ \ \ | | | | | | | | | | | | | | | add tachyon module
| * | | | expose UI port onlyHaoyuan Li2013-11-101-1/+1
| | | | |
| * | | | add tachyon moduleHaoyuan Li2013-11-091-7/+8
| | | | |
* | | | | Merge pull request #157 from rxin/kryoMatei Zaharia2013-11-101-22/+30
|\ \ \ \ \ | |/ / / / |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 3 Kryo related changes. 1. Call Kryo setReferences before calling user specified Kryo registrator. This is done so the user specified registrator can override the default setting. 2. Register more internal classes (MapStatus, BlockManagerId). 3. Slightly refactored the internal class registration to allocate less memory.
| * | | | Moved the Spark internal class registration for Kryo into an object, and ↵Reynold Xin2013-11-091-18/+22
| | | | | | | | | | | | | | | | | | | | added more classes (e.g. MapStatus, BlockManagerId) to the registration.
| * | | | Call Kryo setReferences before calling user specified Kryo registrator.Reynold Xin2013-11-091-6/+10
|/ / / /
* | | | Merge pull request #147 from JoshRosen/fix-java-api-completeness-checkerMatei Zaharia2013-11-093-1/+21
|\ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Add spark-tools assembly to spark-class'ss classpath This commit adds an assembly for `spark-tools` and adds it to `spark-class`'s classpath, allowing the JavaAPICompletenessChecker to be run against Spark 0.8+ with ./spark-class org.apache.spark.tools.JavaAPICompletenessChecker Previously, this tool was run through the `run` script. I chose to add this to `run-example` because I didn't want to duplicate code in a `run-tool` script.
| * | | | Add spark-tools assembly to spark-class classpath.Josh Rosen2013-11-093-1/+21
| | | | | | | | | | | | | | | | | | | | | | | | | This allows the JavaAPICompletenessChecker to be run with Spark 0.8+.
* | | | | Merge pull request #154 from soulmachine/ClusterSchedulerMatei Zaharia2013-11-091-15/+8
|\ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Replace the thread inside ClusterScheduler.start() with an Akka scheduler Threads are precious resources so that we shouldn't abuse them