You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Devaraj K (JIRA)" <ji...@apache.org> on 2017/09/22 22:18:00 UTC

[jira] [Commented] (SPARK-19417) spark.files.overwrite is ignored

    [ https://issues.apache.org/jira/browse/SPARK-19417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16177253#comment-16177253 ] 

Devaraj K commented on SPARK-19417:
-----------------------------------

Thanks [~ckanich] for the test case.
{code:title=SparkContext.scala|borderStyle=solid}
  def addFile(path: String, recursive: Boolean): Unit = {
  ............
    val timestamp = System.currentTimeMillis
    if (addedFiles.putIfAbsent(key, timestamp).isEmpty) {
      logInfo(s"Added file $path at $key with timestamp $timestamp")
      // Fetch the file locally so that closures which are run on the driver can still use the
      // SparkFiles API to access files.
      Utils.fetchFile(uri.toString, new File(SparkFiles.getRootDirectory()), conf,
        env.securityManager, hadoopConfiguration, timestamp, useCache = false)
      postEnvironmentUpdate()
    }
{code}
It is not adding the file if it exists already and it seems to be the intentional behavior, Please find the discussion here https://github.com/apache/spark/pull/14396.

Do you have any real use case to have this?

> spark.files.overwrite is ignored
> --------------------------------
>
>                 Key: SPARK-19417
>                 URL: https://issues.apache.org/jira/browse/SPARK-19417
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 2.1.0
>            Reporter: Chris Kanich
>
> I have not been able to get Spark to actually overwrite a file after I have changed it on the driver node, re-called addFile, and then used it on the executors again. Here's a failing test.
> {code}
>   test("can overwrite files when spark.files.overwrite is true") {
>     val dir = Utils.createTempDir()
>     val file = new File(dir, "file")
>     try {
>       Files.write("one", file, StandardCharsets.UTF_8)
>       sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local-cluster[1,1,1024]")
>          .set("spark.files.overwrite", "true"))
>       sc.addFile(file.getAbsolutePath)
>       def getAddedFileContents(): String = {
>         sc.parallelize(Seq(0)).map { _ =>
>           scala.io.Source.fromFile(SparkFiles.get("file")).mkString
>         }.first()
>       }
>       assert(getAddedFileContents() === "one")
>       Files.write("two", file, StandardCharsets.UTF_8)
>       sc.addFile(file.getAbsolutePath)
>       assert(getAddedFileContents() === "onetwo")
>     } finally {
>       Utils.deleteRecursively(dir)
>       sc.stop()
>     }
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org