Skip to content

[SPARK-6144]When in cluster mode using ADD JAR with a hdfs:// sourced ja... #4881

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

Closed
wants to merge 2 commits into from

Conversation

trystanleftwich
Copy link

...r will fail

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@andrewor14
Copy link
Contributor

Jenkins, this is ok to test.

By the way in the future it would be good to open this against the master branch. In this case it's fine because the 1.3 branch hasn't diverged that much yet.

@SparkQA
Copy link

SparkQA commented Mar 4, 2015

Test build #625 has started for PR 4881 at commit 6e9f069.

  • This patch merges cleanly.

if (!targetDir.mkdir()) {
fileOverwrite: Boolean,
filename: Option[String] = None): Unit = {
if (!targetDir.exists() && !targetDir.mkdir()) {
throw new IOException(s"Failed to create directory ${targetDir.getPath}")
}
fs.listStatus(path).foreach { fileStatus =>
Copy link
Contributor

Choose a reason for hiding this comment

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

To revive the discussion of the previows PR...

Both the problems @andrewor14 and I mentioned exist because of this line. listStatus behaves differently for files and directories, so when path is a directory here you'll get both unwanted behaviors:

  • children will be copied to targetDir instead of $targetDir/${filename.getOrElse(path.getName())}
  • the code will try to copy all children to the target directory with the same filename in the first call from doFetchFile

So this code needs to behave differently depending on whether path is a directory or not. The code Andrew posted here is pretty close to what needs to be done.

@SparkQA
Copy link

SparkQA commented Mar 4, 2015

Test build #625 has finished for PR 4881 at commit 6e9f069.

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

@trystanleftwich
Copy link
Author

So to confirm, i think this function needs to be able to handle 5 states:

Path is a dir which has subdirs
(structure is hdfs://foo/foo1/foo2.jar)
path = hdfs://foo -> local_dir://foo
(Where local_dir://foo looks like local_dir://foo/foo1/foo2.jar)

Path is a file you want to copy with the same name
path = hdfs://foo.jar -> local_dir/foo.jar

Path is a file which you want to copy with a different name
path = hdfs://foo.jar -> local_dir/bar.jar

Path is a dir which contains multiple files that you want to copy with the same names
(Structure is hdfs://foo/foo1.jar, hdfs://foo/foo2.jar)
path = hdfs://foo -> local_dir/foo/
(where local_dir/foo looks like local_dir/foo/foo1.jar and local_dir/foo/foo2.jar)

Path is a dir which contains multiple files and subdirs you want to copy
(similar to above)

Anything I have missed? I have some code in testing now that achieves this ill post it when i've run it on my cluster.

@vanzin
Copy link
Contributor

vanzin commented Mar 4, 2015

I tested with this version of fetchHcfsFile and my tests pass:

/**
 * Fetch a file or directory from a Hadoop-compatible filesystem.
 *
 * Visible for testing
 */
private[spark] def fetchHcfsFile(
    path: Path,
    targetDir: File,
    fs: FileSystem,
    conf: SparkConf,
    hadoopConf: Configuration,
    fileOverwrite: Boolean,
    filename: Option[String] = None): Unit = {
  if (!targetDir.exists() && !targetDir.mkdir()) {
    throw new IOException(s"Failed to create directory ${targetDir.getPath}")
  }
  val dest = new File(targetDir, filename.getOrElse(path.getName))
  if (fs.isFile(path)) {
    val in = fs.open(path)
    try {
      downloadFile(path.toString, in, dest, fileOverwrite)
    } finally {
      in.close()
    }
  } else {
    fs.listStatus(path).foreach { fileStatus =>
      fetchHcfsFile(fileStatus.getPath(), dest, fs, conf, hadoopConf, fileOverwrite)
    }
  }
}

@andrewor14
Copy link
Contributor

@trystanleftwich I believe that's correct. To summarize:

  • Before this patch, adding hdfs://single/file.jar doesn't work (a regression from Spark 1.2)
  • In this patch in its current state, adding hdfs://some/directory/with/multiple/files will not work (not technically a regression from Spark 1.2, but breaks a new feature introduced in SPARK-4687. Add a recursive option to the addFile API #3670)
  • The goal is to make both of these work

The code snippet @vanzin posted should fix both cases.

@trystanleftwich
Copy link
Author

Ok, i've pushed my changes, I've added tests that should cover all the states, I was getting errors with @vanzin code snippet, if you pass in dir i.e path = hdfs://foo and it contains a foo.jar i.e hdfs://foo/foo.jar, The code will create a directory local_dir/foo/foo.jar and not a file local_dir/foo/foo.jar unless im mistaken.

@vanzin
Copy link
Contributor

vanzin commented Mar 4, 2015

The code will create a directory local_dir/foo/foo.jar and not a file

Hmm. Let me check that.

@vanzin
Copy link
Contributor

vanzin commented Mar 4, 2015

Hi @trystanleftwich , just tested my code again with more strict checks, and files show up as files, directories show up as directories.

@andrewor14
Copy link
Contributor

Leaving a link to an alternate fix in #4894

Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false)
assert(targetDir.exists())
assert(targetDir.isDirectory())
// Testing to make sure it doesn't error if the dir already exists
Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false)
val newTempFile = new File(targetDir, tempFile3.getName)
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this is where our patches diverge a bit.

The behavior I expect when I upload a directory "foo", is that there will be a directory called "foo" inside the target directory, so that when I do SparkFiles.get("foo") I get the location of that directory.

Your patch seems to place the contents of "foo" under the target dir, which is not what I'd expect.

Copy link
Author

Choose a reason for hiding this comment

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

Ok, fair enough misunderstanding on my behalf, I've tested your patch and it works in my case, which I expected. I'm happy to close this as likewise I just want this solved for the next release.

@andrewor14
Copy link
Contributor

Let's close this PR in favor of #4894, which I just merged. Thanks for reporting this blocker.

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.

5 participants