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-4896] don’t redundantly overwrite executor JAR deps #2848

Closed
wants to merge 3 commits into from

Conversation

ryan-williams
Copy link
Contributor

No description provided.

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@pwendell
Copy link
Contributor

Jenkins, test this please. This LGTM pending tests - I do wonder whether there is a simple way to structure the logic such that we don't need a big comment explaining the safety there, but I couldn't think of one that didn't involve a lot of nesting.

@SparkQA
Copy link

SparkQA commented Oct 20, 2014

QA tests have started for PR 2848 at commit f3c80ae.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Oct 20, 2014

QA tests have finished for PR 2848 at commit f3c80ae.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21901/
Test PASSed.

@ryan-williams
Copy link
Contributor Author

@preaudc see last commit, I applied this change to the case _ as well, per your suggestion!

@ryan-williams
Copy link
Contributor Author

Jenkins, test this please. Does that work if I am not an admin?

@pwendell agreed, the logic is a little tricky but I couldn't find a simpler way to express it; in the meantime, I factored it out since it was repeated in two cases

@JoshRosen
Copy link
Contributor

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Oct 20, 2014

QA tests have started for PR 2848 at commit 5f1a6f1.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Oct 20, 2014

QA tests have finished for PR 2848 at commit 5f1a6f1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21924/
Test PASSed.

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@ryan-williams
Copy link
Contributor Author

bump; what's the process for getting things merged in? any more comments here?

@ryan-williams
Copy link
Contributor Author

bump. @JoshRosen @pwendell lmk what the process is please?

@pwendell
Copy link
Contributor

At this point its waiting on review from us. No action needed on your side.
I need to look this one over again since it was refactored but I'm sure we
can get it in soon.


Sent from my phone
On Oct 26, 2014 9:49 PM, "Ryan Williams" [email protected] wrote:

bump. @JoshRosen https://github.com/JoshRosen @pwendell
https://github.com/pwendell lmk what the process is please?


Reply to this email directly or view it on GitHub
#2848 (comment).

@ryan-williams
Copy link
Contributor Author

cool, thanks @pwendell. I am new here so don't know if there are other things I'm supposed to be doing!

in: InputStream,
tempFile: File,
destFile: File,
fileOverwrite: Boolean): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

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

style should be

private def moveFile(
    url: String,
    ...
    fileOverwrite: Boolean): Unit = {
  ...
}

@andrewor14
Copy link
Contributor

retest this please, just because Jenkins hasn't tested this one in a while.

@SparkQA
Copy link

SparkQA commented Nov 6, 2014

Test build #23029 has started for PR 2848 at commit 80a20bd.

  • This patch merges cleanly.

@andrewor14
Copy link
Contributor

Hey @ryan-williams +1 on abstracting the duplicate code. Just one clarification: the only change in functionality here is it saves overwriting an existing jar unnecessarily, is that correct? How does this solve SPARK-3967?

@ryan-williams
Copy link
Contributor Author

Thanks for having a look, I'll get to that style nit momentarily.

The tie-in to SPARK-3967 is that, on my cluster, when attempting to redundantly copy the JAR deps, I was getting "Permission Denied" errors because not only were the JARs already there, they were read-only and could not be overwritten.

I did not get to the bottom of who was putting them there; it seemed like every executor would try to copy its JARs, find them to be already present, try to copy anyway, and fail due to "permission denied."

Simply skipping the redundant copy attempt (as this PR does) resulted in my jobs passing, and it seemed like the original authors intended to not attempt to redundantly copy but had a bug, so I settled for this "fix" and didn't get fully to the bottom of this.

For example, suppose an executor E hit the code path where a given JAR is present but its contents are not identical to the one E wants. In this case, E would attempt to overwrite the existing JAR and fail due to the same "Permission denied", so there could be some larger way in which it needs to be made possible for executors to overwrite their JAR deps (maybe that is just a cluster config issue I and @preaudc are hitting?), but if so I think that's orthogonal to this change.

Let me know if I should just take the "SPARK-3967" label out of this PR, since it might not fully "solve" the issue.

@ryan-williams
Copy link
Contributor Author

fixed the style nit, also added a comment nit I found in CoGroupedRDD, lmk if you want that in a separate PR or something

@SparkQA
Copy link

SparkQA commented Nov 7, 2014

Test build #23029 has finished for PR 2848 at commit 80a20bd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/23029/
Test PASSed.

}
// If the destFile exists at this point, it is equal to tempFile and we can skip moving
// it; the code above deletes it or throws an Exception, as appropriate, if it exists and
// is not equal to tempFile.
Copy link
Contributor

Choose a reason for hiding this comment

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

Hey @ryan-williams my concern is that if the delete fails (e.g. because of some permissions thing), then we will always have an old version of the file because destFile.exists will always be true. It might make sense to throw an exception if delete is unsuccessful instead of logging a message up there in L429 that is technically not true (I realize you just moved the code, but it would still be good to fix this).

@andrewor14
Copy link
Contributor

Hey @ryan-williams thanks for your explanation. Looks like there could be another potential source of the same issue in L397, when we use the executor cache.

@ryan-williams
Copy link
Contributor Author

OK, I refactored a little further. Updates:

  • renamed the helper function maybeMoveFile (instead of moveFile)
  • introduced a second signature for maybeMoveFile that just takes two Files
    • this allowed me to bring the 3rd instance of this repeated logic in Utils.doFetchFile into the fold, which helps the overall consistency / cleanliness a lot, I think.
    • incidentally, that last code path handled the exists vs. delete() trickery differently than I was doing before; it used a boolean var that recorded explicitly whether we shouldCopy (true to start, set to false iff we found an identical file to exist). I decided that this way was cleaner, per @andrewor14's and @pwendell's (earlier in this thread) suggestions, and structured maybeMoveFile that way.
  • folded the code path around L397 into maybeMoveFile as well, per @andrewor14's last suggestion.

lmk how it looks!

@ryan-williams
Copy link
Contributor Author

OK, I added logic for making sure we always delete tempFile and throw an exception if deletion of an existing sourceFile fails; I think I hit all the other feedback as well.

@JoshRosen
Copy link
Contributor

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Dec 13, 2014

Test build #24431 has started for PR 2848 at commit bcf2543.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Dec 13, 2014

Test build #24431 has finished for PR 2848 at commit bcf2543.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24431/
Test FAILed.

@JoshRosen
Copy link
Contributor

Hmm, maybe a legitimate test failure?

[info] - Distributing files locally security On *** FAILED *** (82 milliseconds)
[info]   java.io.FileNotFoundException: /tmp/spark-27c67e3e-d5f5-4d86-b9fa-948319ea6b7a/test/FileServerSuite.txt (No such file or directory)
[info]   at java.io.FileInputStream.open(Native Method)
[info]   at java.io.FileInputStream.<init>(FileInputStream.java:146)
[info]   at com.google.common.io.Files$FileByteSource.openStream(Files.java:127)
[info]   at com.google.common.io.Files$FileByteSource.openStream(Files.java:117)
[info]   at com.google.common.io.ByteSource.copyTo(ByteSource.java:247)
[info]   at com.google.common.io.Files.copy(Files.java:458)
[info]   at org.apache.spark.HttpFileServer.addFileToDir(HttpFileServer.scala:72)
[info]   at org.apache.spark.HttpFileServer.addFile(HttpFileServer.scala:55)
[info]   at org.apache.spark.SparkContext.addFile(SparkContext.scala:965)
[info]   at org.apache.spark.FileServerSuite$$anonfun$3.apply$mcV$sp(FileServerSuite.scala:96)
[info]   at org.apache.spark.FileServerSuite$$anonfun$3.apply(FileServerSuite.scala:90)
[info]   at org.apache.spark.FileServerSuite$$anonfun$3.apply(FileServerSuite.scala:90)
[info]   at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
[info]   at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
[info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
[info]   at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
[info]   at org.scalatest.Suite$class.withFixture(Suite.scala:1122)
[info]   at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555)
[info]   at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
[info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[...]

@ryan-williams
Copy link
Contributor Author

interesting. this may be more appropriate on my gigantic thread about testing/development best practices, but do you have any recommendations for debugging this?

AFAICT, this seems to be a test that requires a full assembly-build to run; I've tried running mvn -Dsuites=FileServerSuite test on a branch cut from the same master that this PR is cut from (b004150) and I get strange failure modes, usually just that the test seems to hang for hours. I ran a mvn package -DskipTests to build a fresh assembly, and tried again, and it still hung. I'm currently waiting on a mvn clean package -DskipTests, at which point I will try again. Debugging / iterating on a particular (integration) test, like I need to do here, remains a prohibitively slow process for me.

I'm guessing this failure must have something to do with having moved / removed (temp?) files that this test was relying on; my naive approach will be to comment out some of the move/removal lines and see if that fixes the test, but on pain of running a mvn clean package -DskipTests every time.

Is there a better way?

@JoshRosen
Copy link
Contributor

Disclaimer: for iterative debugging, I use sbt to build Spark, not Maven. Spark 1.2.0 has docs on building with SBT. If possible, I'd switch to using that workflow.

The issue here is probably that you're running a full mvn clean and starting over from scratch after each change. I'd like to help move this PR along, so the following is going to be some interactive logs to see whether I can quickly iterate on this using Maven.

Let's say that I'm starting from a completely cold build (but with Maven dependencies already downloaded):

# Since I have zinc installed, I'll use it:
zinc -start
git checkout /a/branch/with/your/pr/code
# Here, the -T C1 says "build in parallel with one thread per core":
time mvn -T C1 clean package -DskipTests

This didn't take super long, but it was a few minutes:

real    4m19.537s
user    3m14.634s
sys 0m16.882s

Let's run just the test suite that we're interested in (instructions from here:

time mvn -T C1 test -DwildcardSuites=org.apache.spark.FileServerSuite

This took a little while because it had to build a bunch of test sources, but it was only a few seconds before the tests started running (and failing):

real    0m33.968s
user    0m36.544s
sys 0m3.032s

FileServerSuite:
- Distributing files locally
- Distributing files locally security On *** FAILED ***
  java.io.FileNotFoundException: /var/folders/0k/2qp2p2vs7bv033vljnb8nk1c0000gn/T/spark-7206dcdd-9db6-4d78-ac9e-2de69619dc67/test/FileServerSuite.txt (No such file or directory)
  at java.io.FileInputStream.open(Native Method)
  at java.io.FileInputStream.<init>(FileInputStream.java:146)
  at com.google.common.io.Files$FileByteSource.openStream(Files.java:124)
  at com.google.common.io.Files$FileByteSource.openStream(Files.java:114)
  at com.google.common.io.ByteSource.copyTo(ByteSource.java:202)
  at com.google.common.io.Files.copy(Files.java:436)
  at org.apache.spark.HttpFileServer.addFileToDir(HttpFileServer.scala:72)
  at org.apache.spark.HttpFileServer.addFile(HttpFileServer.scala:55)
  at org.apache.spark.SparkContext.addFile(SparkContext.scala:965)
  at org.apache.spark.FileServerSuite$$anonfun$3.apply$mcV$sp(FileServerSuite.scala:96)
  ...
- Distributing files locally using URL as input *** FAILED ***
  java.io.FileNotFoundException: /var/folders/0k/2qp2p2vs7bv033vljnb8nk1c0000gn/T/spark-7206dcdd-9db6-4d78-ac9e-2de69619dc67/test/FileServerSuite.txt (No such file or directory)
  at java.io.FileInputStream.open(Native Method)
  at java.io.FileInputStream.<init>(FileInputStream.java:146)
  at com.google.common.io.Files$FileByteSource.openStream(Files.java:124)
  at com.google.common.io.Files$FileByteSource.openStream(Files.java:114)
  at com.google.common.io.ByteSource.copyTo(ByteSource.java:202)
  at com.google.common.io.Files.copy(Files.java:436)
  at org.apache.spark.HttpFileServer.addFileToDir(HttpFileServer.scala:72)
  at org.apache.spark.HttpFileServer.addFile(HttpFileServer.scala:55)
  at org.apache.spark.SparkContext.addFile(SparkContext.scala:965)
  at org.apache.spark.FileServerSuite$$anonfun$5.apply$mcV$sp(FileServerSuite.scala:112)
  ...
 - Dynamically adding JARS locally
- Distributing files on a standalone cluster *** FAILED ***
  java.io.FileNotFoundException: /var/folders/0k/2qp2p2vs7bv033vljnb8nk1c0000gn/T/spark-7206dcdd-9db6-4d78-ac9e-2de69619dc67/test/FileServerSuite.txt (No such file or directory)
  at java.io.FileInputStream.open(Native Method)
  at java.io.FileInputStream.<init>(FileInputStream.java:146)
  at com.google.common.io.Files$FileByteSource.openStream(Files.java:124)
  at com.google.common.io.Files$FileByteSource.openStream(Files.java:114)
  at com.google.common.io.ByteSource.copyTo(ByteSource.java:202)
  at com.google.common.io.Files.copy(Files.java:436)
  at org.apache.spark.HttpFileServer.addFileToDir(HttpFileServer.scala:72)
  at org.apache.spark.HttpFileServer.addFile(HttpFileServer.scala:55)
  at org.apache.spark.SparkContext.addFile(SparkContext.scala:965)
  at org.apache.spark.FileServerSuite$$anonfun$8.apply$mcV$sp(FileServerSuite.scala:137)
  ...
- Dynamically adding JARS on a standalone cluster
- Dynamically adding JARS on a standalone cluster using local: URL

Let's try adding a print statement to SparkContext.addFile, then re-running the tests. We could do this by re-packaging:

mvn -T C1 package -DskipTests
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time: 01:24 min (Wall Clock)
[INFO] Finished at: 2014-12-15T18:47:00-08:00
[INFO] Final Memory: 50M/1535M

And to re-run the test:

time mvn -T C1 test -DwildcardSuites=org.apache.spark.FileServerSuite
real    0m40.149s
user    0m35.894s
sys 0m3.163s

So this took a couple of minutes.

Let's do the same thing in SBT. First, let's start off from a completely clean-slate:

time sbt/sbt clean package assembly

The timing here could be messed up because I have a multi-core machine:

real    3m53.643s
user    8m22.337s
sys 1m15.794s

Next, let's run just the suite we're interested in. Here's a naive way to do this, which involves building every test, so this first run will take longer than subsequent runs:

time sbt/sbt "test-only FileServerSuite"
[success] Total time: 88 s, completed Dec 15, 2014 6:56:35 PM

real    1m39.013s
user    7m37.323s
sys 0m16.206s

Whoops, I made a mistake here! My test-only pattern didn't include a wildcard, so FileServerSuite didn't match the fully-qualified name of a test suite. Let me go ahead and re-run with the right command:

time sbt/sbt "test-only *FileServerSuite"

This was pretty fast:

real    0m29.075s
user    0m50.744s
sys 0m3.512s

I could also have run this from the interactive shell to get automatic rebuilding on source changes.

There's an even faster way of running just FileServerSuite, though: I can tell SBT to only build / run the core module. This time, let's do this interactively, but from a clean slate:

[info] Set current project to spark-parent (in build file:/Users/joshrosen/Documents/spark/)
> clean
[success] Total time: 20 s, completed Dec 15, 2014 7:01:20 PM
> project core
[info] Set current project to spark-core (in build file:/Users/joshrosen/Documents/spark/)
> package
[...]
[info] Compiling 42 Java sources to /Users/joshrosen/Documents/spark/network/common/target/scala-2.10/classes...
[info] Compiling 20 Java sources to /Users/joshrosen/Documents/spark/network/shuffle/target/scala-2.10/classes...
[info] Compiling 397 Scala sources and 33 Java sources to /Users/joshrosen/Documents/spark/core/target/scala-2.10/classes...
[...]
[info] Packaging /Users/joshrosen/Documents/spark/core/target/scala-2.10/spark-core_2.10-1.3.0-SNAPSHOT.jar ...
[info] Done packaging.
[success] Total time: 64 s, completed Dec 15, 2014 7:02:36 PM
> test-only *FileServerSuite
[...]
[info] Compiling 124 Scala sources and 4 Java sources to /Users/joshrosen/Documents/spark/core/target/scala-2.10/test-classes...
[...]
[---- tests run ----]
[---- tests go into infinite loop ----]
ls: /Users/joshrosen/Documents/spark/assembly/target/scala-2.10: No such file or directory
ls: /Users/joshrosen/Documents/spark/assembly/target/scala-2.10: No such file or directory
ls: /Users/joshrosen/Documents/spark/assembly/target/scala-2.10: No such file or directory
ls: /Users/joshrosen/Documents/spark/assembly/target/scala-2.10: No such file or directory
[... infinite repetitions ...]

Hmm, so it looks like the tests that rely on local-cluster mode need to have access to a Spark assembly JAR in order to run, and that there's mis-handling of this error condition somewhere (hence the infinite loop). This is pretty annoying, so I guess I'll build an assembly once then use export SPARK_PREPEND_CLASSES=true so that I don't have to keep re-building it across test runs:

export SPARK_PREPEND_CLASSES=true
sbt/sbt assembly/assembly
sbt/sbt

Now, from the SBT shell:

> project core
[info] Set current project to spark-core (in build file:/Users/joshrosen/Documents/spark/)
> ~test-only *FileServerSuite
[... tests run ...]
[info] *** 3 TESTS FAILED ***
[error] Failed: Total 7, Failed 3, Errors 0, Passed 4
[error] Failed tests:
[error]     org.apache.spark.FileServerSuite
[error] (core/test:testOnly) sbt.TestsFailedException: Tests unsuccessful
[error] Total time: 18 s, completed Dec 15, 2014 7:10:57 PM
1. Waiting for source changes... (press enter to interrupt)
[... add a println to Utils.addFile ...]
[... tests start up almost instantly and run ...]
[info] *** 3 TESTS FAILED ***
[error] Failed: Total 7, Failed 3, Errors 0, Passed 4
[error] Failed tests:
[error]     org.apache.spark.FileServerSuite
[error] (core/test:testOnly) sbt.TestsFailedException: Tests unsuccessful
[error] Total time: 19 s, completed Dec 15, 2014 7:11:46 PM

So, to summarize: I agree that there are a bunch of pain-points in the current build process. Day-to-day, though, it hasn't affected me that much since I'll usually sbt/sbt clean package assembly and export SPARK_PREPEND_CLASSES=true once at the beginning of the day then keep working in my SBT shell, where incremental recompilation means that I can make changes in my IDE and see the failing test update (almost) instantly.

@ryan-williams
Copy link
Contributor Author

Thanks a lot for the build-process brain-dump, @JoshRosen; some of that is way better than processes I've wasted a lot of time on. I'm interested in getting all of these tips and tricks pushed to Spark docs, but even after all of the emailing your response includes tricks I've not seen documented anywhere (e.g. SPARK_PREPEND_CLASSES), so I'm leery of spending time trying to document what I've learned for fear that I'm still not at the bottom of the rabbit hole.

In any case, I was able to debug and (I believe) fix the test failure, which was a result of having inadvertently changed a file "copy" to a "move", so it's a good thing the test caught it.

This should be ready to re-test.

@JoshRosen
Copy link
Contributor

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Dec 17, 2014

Test build #24518 has started for PR 2848 at commit c14daff.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Dec 17, 2014

Test build #24518 has finished for PR 2848 at commit c14daff.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24518/
Test PASSed.

@JoshRosen
Copy link
Contributor

This looks good to me, so I'm ready to merge it. One final little thing, though: since we have two separate PRs that address SPARK-3967, I'd like to create a new JIRA for this fix just in case this doesn't fix the YARN issue in the original ticket.

Therefore, I've opened https://issues.apache.org/jira/browse/SPARK-4896. Do you mind editing this PR's title to reference that JIRA instead? Once you do that, I'll merge this into all of the maintenance branches and resolve the original JIRA.

Thanks for your patience during this long review and with the build difficulties. I think that the end result here is a significant improvement over the old fetchFile code, so I'm looking forward to merging this.

@ryan-williams ryan-williams changed the title [SPARK-3967] don’t redundantly overwrite executor JAR deps [SPARK-4896] don’t redundantly overwrite executor JAR deps Dec 19, 2014
@ryan-williams
Copy link
Contributor Author

thanks for shepherding this @JoshRosen, I've updated the PR title accordingly, lmk if I should do anything else here!

@JoshRosen
Copy link
Contributor

Thanks for updating the title.

This looks good to me, so I'm going to merge this into master (1.3.0), branch-1.2 (1.2.1), and branch-1.1 (1.1.2).

asfgit pushed a commit that referenced this pull request Dec 19, 2014
Author: Ryan Williams <[email protected]>

Closes #2848 from ryan-williams/fetch-file and squashes the following commits:

c14daff [Ryan Williams] Fix copy that was changed to a move inadvertently
8e39c16 [Ryan Williams] code review feedback
788ed41 [Ryan Williams] don’t redundantly overwrite executor JAR deps

(cherry picked from commit 7981f96)
Signed-off-by: Josh Rosen <[email protected]>
@asfgit asfgit closed this in 7981f96 Dec 19, 2014
@JoshRosen
Copy link
Contributor

Looks like the branch-1.1 backport is going to be a little tricky due to merge conflicts because #1843 wasn't present in that branch so the Hadoop configuration is passed in a little differently. I'll try to fix up the conflict myself and let Jenkins test it on merge.

@JoshRosen
Copy link
Contributor

Oh, and also because some of the cache stuff wasn't present.

asfgit pushed a commit that referenced this pull request Dec 19, 2014
Author: Ryan Williams <[email protected]>

Closes #2848 from ryan-williams/fetch-file and squashes the following commits:

c14daff [Ryan Williams] Fix copy that was changed to a move inadvertently
8e39c16 [Ryan Williams] code review feedback
788ed41 [Ryan Williams] don’t redundantly overwrite executor JAR deps

(cherry picked from commit 7981f96)
Signed-off-by: Josh Rosen <[email protected]>

Conflicts:
	core/src/main/scala/org/apache/spark/util/Utils.scala
@JoshRosen
Copy link
Contributor

Alright, finished the 1.1 cherry-pick: 546a239

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants