Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-2546

Configuration object thread safety issue

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: 0.9.1, 1.0.2, 1.1.0, 1.2.0
    • Fix Version/s: 1.0.3, 1.1.1, 1.2.0
    • Component/s: Spark Core
    • Labels:
      None
    • Target Version/s:

      Description

      // observed in 0.9.1 but expected to exist in 1.0.1 as well

      This ticket is copy-pasted from a thread on the dev@ list:

      We discovered a very interesting bug in Spark at work last week in Spark 0.9.1 — that the way Spark uses the Hadoop Configuration object is prone to thread safety issues. I believe it still applies in Spark 1.0.1 as well. Let me explain:

      Observations

      • Was running a relatively simple job (read from Avro files, do a map, do another map, write back to Avro files)
      • 412 of 413 tasks completed, but the last task was hung in RUNNING state
      • The 412 successful tasks completed in median time 3.4s
      • The last hung task didn't finish even in 20 hours
      • The executor with the hung task was responsible for 100% of one core of CPU usage
      • Jstack of the executor attached (relevant thread pasted below)

      Diagnosis

      After doing some code spelunking, we determined the issue was concurrent use of a Configuration object for each task on an executor. In Hadoop each task runs in its own JVM, but in Spark multiple tasks can run in the same JVM, so the single-threaded access assumptions of the Configuration object no longer hold in Spark.

      The specific issue is that the AvroRecordReader actually modifies the JobConf it's given when it's instantiated! It adds a key for the RPC protocol engine in the process of connecting to the Hadoop FileSystem. When many tasks start at the same time (like at the start of a job), many tasks are adding this configuration item to the one Configuration object at once. Internally Configuration uses a java.lang.HashMap, which isn't threadsafe… The below post is an excellent explanation of what happens in the situation where multiple threads insert into a HashMap at the same time.

      http://mailinator.blogspot.com/2009/06/beautiful-race-condition.html

      The gist is that you have a thread following a cycle of linked list nodes indefinitely. This exactly matches our observations of the 100% CPU core and also the final location in the stack trace.

      So it seems the way Spark shares a Configuration object between task threads in an executor is incorrect. We need some way to prevent concurrent access to a single Configuration object.

      Proposed fix

      We can clone the JobConf object in HadoopRDD.getJobConf() so each task gets its own JobConf object (and thus Configuration object). The optimization of broadcasting the Configuration object across the cluster can remain, but on the other side I think it needs to be cloned for each task to allow for concurrent access. I'm not sure the performance implications, but the comments suggest that the Configuration object is ~10KB so I would expect a clone on the object to be relatively speedy.

      Has this been observed before? Does my suggested fix make sense? I'd be happy to file a Jira ticket and continue discussion there for the right way to fix.

      Thanks!
      Andrew

      P.S. For others seeing this issue, our temporary workaround is to enable spark.speculation, which retries failed (or hung) tasks on other machines.

      "Executor task launch worker-6" daemon prio=10 tid=0x00007f91f01fe000 nid=0x54b1 runnable [0x00007f92d74f1000]
         java.lang.Thread.State: RUNNABLE
          at java.util.HashMap.transfer(HashMap.java:601)
          at java.util.HashMap.resize(HashMap.java:581)
          at java.util.HashMap.addEntry(HashMap.java:879)
          at java.util.HashMap.put(HashMap.java:505)
          at org.apache.hadoop.conf.Configuration.set(Configuration.java:803)
          at org.apache.hadoop.conf.Configuration.set(Configuration.java:783)
          at org.apache.hadoop.conf.Configuration.setClass(Configuration.java:1662)
          at org.apache.hadoop.ipc.RPC.setProtocolEngine(RPC.java:193)
          at org.apache.hadoop.hdfs.NameNodeProxies.createNNProxyWithClientProtocol(NameNodeProxies.java:343)
          at org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:168)
          at org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:129)
          at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:436)
          at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:403)
          at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:125)
          at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2262)
          at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:86)
          at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2296)
          at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2278)
          at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:316)
          at org.apache.hadoop.fs.Path.getFileSystem(Path.java:194)
          at org.apache.avro.mapred.FsInput.<init>(FsInput.java:37)
          at org.apache.avro.mapred.AvroRecordReader.<init>(AvroRecordReader.java:43)
          at org.apache.avro.mapred.AvroInputFormat.getRecordReader(AvroInputFormat.java:52)
          at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:156)
          at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:149)
          at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:64)
          at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:241)
          at org.apache.spark.rdd.RDD.iterator(RDD.scala:232)
          at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
          at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:241)
          at org.apache.spark.rdd.RDD.iterator(RDD.scala:232)
          at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
          at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:241)
          at org.apache.spark.rdd.RDD.iterator(RDD.scala:232)
          at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:109)
          at org.apache.spark.scheduler.Task.run(Task.scala:53)
          at org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:211)
          at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:42)
          at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:41)
          at java.security.AccessController.doPrivileged(Native Method)
          at javax.security.auth.Subject.doAs(Subject.java:415)
          at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
          at org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:41)
          at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:176)
          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
          at java.lang.Thread.run(Thread.java:745)
      

        Issue Links

          Activity

          Hide
          aash Andrew Ash added a comment -

          On the thread:

          Me:

          Reynold's recent announcement of the broadcast RDD object patch may also have implications of the right path forward here. I'm not sure I fully understand the implications though: https://github.com/apache/spark/pull/1452

          "Once this is committed, we can also remove the JobConf broadcast in HadoopRDD."

          Patrick Wendell:

          I think you are correct and a follow up to SPARK-2521 will end up
          fixing this. The desing of SPARK-2521 automatically broadcasts RDD
          data in tasks and the approach creates a new copy of the RDD and
          associated data for each task. A natural follow-up to that patch is to
          stop handling the jobConf separately (since we will now broadcast all
          referents of the RDD itself) and just have it broadcasted with the
          RDD. I'm not sure if Reynold plans to include this in SPARK-2521 or
          afterwards, but it's likely we'd do that soon.

          Show
          aash Andrew Ash added a comment - On the thread: Me: Reynold's recent announcement of the broadcast RDD object patch may also have implications of the right path forward here. I'm not sure I fully understand the implications though: https://github.com/apache/spark/pull/1452 "Once this is committed, we can also remove the JobConf broadcast in HadoopRDD." Patrick Wendell : I think you are correct and a follow up to SPARK-2521 will end up fixing this. The desing of SPARK-2521 automatically broadcasts RDD data in tasks and the approach creates a new copy of the RDD and associated data for each task. A natural follow-up to that patch is to stop handling the jobConf separately (since we will now broadcast all referents of the RDD itself) and just have it broadcasted with the RDD. I'm not sure if Reynold plans to include this in SPARK-2521 or afterwards, but it's likely we'd do that soon.
          Hide
          pwendell Patrick Wendell added a comment -

          Ideally we should merge either this or SPARK-2585 in the 1.1 release.

          Show
          pwendell Patrick Wendell added a comment - Ideally we should merge either this or SPARK-2585 in the 1.1 release.
          Hide
          joshrosen Josh Rosen added a comment -

          Hi Andrew,

          Do you have any way to reliably reproduce this issue? I'm considering implementing a clone()-based approach and I'd like to have a way to test whether I've fixed this bug.

          Show
          joshrosen Josh Rosen added a comment - Hi Andrew, Do you have any way to reliably reproduce this issue? I'm considering implementing a clone()-based approach and I'd like to have a way to test whether I've fixed this bug.
          Hide
          ash211 Andrew Ash added a comment -

          I don't have a reliable repro that's in a unit test format. On my prod
          cluster though it reproduces quite reliably! I'd suggest using the
          AvroInputFormat on sizable files and a large number of partitions – I had
          O(400) partitions and O(15GB) of data in that dataset.

          Sidenote – the trouble with unit testing race conditions is that you have
          to run them for a long time in an error-prone situation and hope that the
          behavior is triggered. You could verify that the Configuration objects
          each partition gets are equal() but not reference equal, but that's not
          directly testing for the race condition.

          Show
          ash211 Andrew Ash added a comment - I don't have a reliable repro that's in a unit test format. On my prod cluster though it reproduces quite reliably! I'd suggest using the AvroInputFormat on sizable files and a large number of partitions – I had O(400) partitions and O(15GB) of data in that dataset. Sidenote – the trouble with unit testing race conditions is that you have to run them for a long time in an error-prone situation and hope that the behavior is triggered. You could verify that the Configuration objects each partition gets are equal() but not reference equal, but that's not directly testing for the race condition.
          Hide
          pwendell Patrick Wendell added a comment -

          Hey Andrew I think due to us cutting SPARK-2585 from this release it will remain broken in Spark 1.1. We could look into a solution based on clone()'ing the conf for future patch releases in the 1.1 branch.

          Show
          pwendell Patrick Wendell added a comment - Hey Andrew I think due to us cutting SPARK-2585 from this release it will remain broken in Spark 1.1. We could look into a solution based on clone()'ing the conf for future patch releases in the 1.1 branch.
          Hide
          ash211 Andrew Ash added a comment -

          Ok I'll stay on the lookout for this bug and ping here again if we observe
          this. Luckily we haven't seen this particular issue since, but that's
          mostly been because other things are causing problems.

          We have a few bugs now that are nondeterministically broken in Spark and
          cause jobs to fail/hang, but if we retry the job several times (and
          spark.speculation helps somewhat) we can usually eventually get a job to
          complete. I can share that list if you're interested of what's highest on
          our minds right now.

          On Fri, Aug 15, 2014 at 6:03 PM, Patrick Wendell (JIRA) <jira@apache.org>

          Show
          ash211 Andrew Ash added a comment - Ok I'll stay on the lookout for this bug and ping here again if we observe this. Luckily we haven't seen this particular issue since, but that's mostly been because other things are causing problems. We have a few bugs now that are nondeterministically broken in Spark and cause jobs to fail/hang, but if we retry the job several times (and spark.speculation helps somewhat) we can usually eventually get a job to complete. I can share that list if you're interested of what's highest on our minds right now. On Fri, Aug 15, 2014 at 6:03 PM, Patrick Wendell (JIRA) <jira@apache.org>
          Hide
          aash Andrew Ash added a comment -

          Another proposed fix: extend JobConf as a shim and replace the Hadoop one with one that's threadsafe

          Show
          aash Andrew Ash added a comment - Another proposed fix: extend JobConf as a shim and replace the Hadoop one with one that's threadsafe
          Hide
          joshrosen Josh Rosen added a comment -

          JobConf has a ton of methods and it's not clear whether we can get away with synchronizing only some of them.

          I'm going to look into using Scala macro annotations (http://docs.scala-lang.org/overviews/macros/annotations.html) to create a @synchronizeAll macro for adding synchronization to all methods of a class.

          Show
          joshrosen Josh Rosen added a comment - JobConf has a ton of methods and it's not clear whether we can get away with synchronizing only some of them. I'm going to look into using Scala macro annotations ( http://docs.scala-lang.org/overviews/macros/annotations.html ) to create a @synchronizeAll macro for adding synchronization to all methods of a class.
          Hide
          joshrosen Josh Rosen added a comment -

          A synchronization wrapper (whether written by hand or generated using macros) might introduce an unwanted runtime dependency on the exact compile-time version of Hadoop that we used. For example, say we compile against Hadoop 1.x and run on Hadoop 1.y (where y > x) and the runtime version of JobConf contains methods that were not present in the version that we wrapped at compile-time. What happens in this case?

          Before we explore this option, I should probably re-visit SPARK-2585 to see if I can understand why the patch seemed to introduce a performance regression, since that approach is Hadoop version agnostic.

          Show
          joshrosen Josh Rosen added a comment - A synchronization wrapper (whether written by hand or generated using macros) might introduce an unwanted runtime dependency on the exact compile-time version of Hadoop that we used. For example, say we compile against Hadoop 1.x and run on Hadoop 1.y (where y > x) and the runtime version of JobConf contains methods that were not present in the version that we wrapped at compile-time. What happens in this case? Before we explore this option, I should probably re-visit SPARK-2585 to see if I can understand why the patch seemed to introduce a performance regression, since that approach is Hadoop version agnostic.
          Hide
          joshrosen Josh Rosen added a comment -

          I've decided to go with the cloning approach, since this seems simplest and safest.

          It looks like SparkContext has a public hadoopConfiguration val that holds a re-used Configuration object. It looks like this may have been purposely exposed to allow users to set Hadoop configuration properties (see how it's mentioned in docs/storage-openstack-swift.md; the Spark EC2 instructions also mention using this attribute to set S3 credentials). This object is used as the default Hadoop configuration in the newAPIHadoopRDD and saveAsHadoop* methods; it's also read in many other places inside of Spark.

          While SPARK-2585 addressed sharing of the Configuration objects in executors, it seems that we still might face races in the driver if multiple threads are sharing a SparkContext and one thread mutates the shared configuration while another thread submits a job that reads it.

          This seems like a tricky problem to fix. I don't think that we can change SparkContext.hadoopConfiguration to return a copy of the configuration object, since it seems that the shared / mutating semantics are required by some existing code. At the same time, we can't simply clone the return value before using it in our internal driver-side code since a) we can't lock out writers/mutators while performing the clone() and b) the change in semantics might break existing user-code. Essentially, I don't think that there's anything that we can do that's guaranteed to be safe once a Configuration has been exposed to multiple threads; we need to perform the cloning before the object has been shared.

          Show
          joshrosen Josh Rosen added a comment - I've decided to go with the cloning approach, since this seems simplest and safest. It looks like SparkContext has a public hadoopConfiguration val that holds a re-used Configuration object. It looks like this may have been purposely exposed to allow users to set Hadoop configuration properties (see how it's mentioned in docs/storage-openstack-swift.md; the Spark EC2 instructions also mention using this attribute to set S3 credentials). This object is used as the default Hadoop configuration in the newAPIHadoopRDD and saveAsHadoop* methods; it's also read in many other places inside of Spark. While SPARK-2585 addressed sharing of the Configuration objects in executors, it seems that we still might face races in the driver if multiple threads are sharing a SparkContext and one thread mutates the shared configuration while another thread submits a job that reads it. This seems like a tricky problem to fix. I don't think that we can change SparkContext.hadoopConfiguration to return a copy of the configuration object, since it seems that the shared / mutating semantics are required by some existing code. At the same time, we can't simply clone the return value before using it in our internal driver-side code since a) we can't lock out writers/mutators while performing the clone() and b) the change in semantics might break existing user-code. Essentially, I don't think that there's anything that we can do that's guaranteed to be safe once a Configuration has been exposed to multiple threads; we need to perform the cloning before the object has been shared.
          Hide
          joshrosen Josh Rosen added a comment -

          Here are a few "in the wild" examples of how sc.hadoopConfiguration is currently used, to give a sense of the impact of any changes that we might make here.

          Setting elasticserach configuration properties:

          sc.hadoopConfiguration.set("es.resource", "syslog/entry")
          output.saveAsHadoopFile[ESOutputFormat]("-")
          

          Setting S3 credentials:

          val conf = new SparkConf().setAppName("Simple Application").setMaster("local")      
          val sc = new SparkContext(conf)
          val hadoopConf=sc.hadoopConfiguration;
          hadoopConf.set("fs.s3.impl", "org.apache.hadoop.fs.s3native.NativeS3FileSystem")
          hadoopConf.set("fs.s3.awsAccessKeyId",myAccessKey)
          hadoopConf.set("fs.s3.awsSecretAccessKey",mySecretKey)
          

          There's a lot more examples here: https://github.com/search?utf8=%E2%9C%93&q=%22sc.hadoopconfiguration%22&type=Code&ref=searchresults

          The most common use-case seems to be setting S3 credentials. One option would be to slowly deprecate the existing hadoopConfiguration field in favor of methods for setting S3 credentials. Currently, you can set these options in SparkConf before creating the SparkContext; unfortunately, this isn't an option for users that want to set configurations after starting SparkContext (e.g. IPython notebook users). I suppose that these users could work with a clone of the configuration object and manually pass that object into Spark methods.

          If we did add a SparkContext-wide setting for changing Hadoop configurations, then in multi-user shared-SparkContext environments we run the risk of users overwriting each others' S3 credentials.

          Show
          joshrosen Josh Rosen added a comment - Here are a few "in the wild" examples of how sc.hadoopConfiguration is currently used, to give a sense of the impact of any changes that we might make here. Setting elasticserach configuration properties : sc.hadoopConfiguration.set( "es.resource" , "syslog/entry" ) output.saveAsHadoopFile[ESOutputFormat]( "-" ) Setting S3 credentials : val conf = new SparkConf().setAppName( "Simple Application" ).setMaster( "local" ) val sc = new SparkContext(conf) val hadoopConf=sc.hadoopConfiguration; hadoopConf.set( "fs.s3.impl" , "org.apache.hadoop.fs.s3native.NativeS3FileSystem" ) hadoopConf.set( "fs.s3.awsAccessKeyId" ,myAccessKey) hadoopConf.set( "fs.s3.awsSecretAccessKey" ,mySecretKey) There's a lot more examples here: https://github.com/search?utf8=%E2%9C%93&q=%22sc.hadoopconfiguration%22&type=Code&ref=searchresults The most common use-case seems to be setting S3 credentials. One option would be to slowly deprecate the existing hadoopConfiguration field in favor of methods for setting S3 credentials. Currently, you can set these options in SparkConf before creating the SparkContext; unfortunately, this isn't an option for users that want to set configurations after starting SparkContext (e.g. IPython notebook users). I suppose that these users could work with a clone of the configuration object and manually pass that object into Spark methods. If we did add a SparkContext-wide setting for changing Hadoop configurations, then in multi-user shared-SparkContext environments we run the risk of users overwriting each others' S3 credentials.
          Hide
          joshrosen Josh Rosen added a comment -

          For now, let's ignore the design issue of whether the current API is confusing in multi-user shared-SparkContext environments. If we want to keep the current API without any driver-side thread-safety issues, is there anything that we can do?

          Maybe we can add a very limited amount of synchronization to Configuration. Looking at a recent version of Configuration.java, it seems that the private updatingResource HashMap and finalParameters HashSet fields the only non-thread-safe collections in Configuration (Java's Properties class is thread-safe).

          My hunch is that the updatingResource HashMap was the map referred to by the stacktrace posted in this issue. We might be able to use reflection to find this field and inject a synchronized HashMap instead.

          Show
          joshrosen Josh Rosen added a comment - For now, let's ignore the design issue of whether the current API is confusing in multi-user shared-SparkContext environments. If we want to keep the current API without any driver-side thread-safety issues, is there anything that we can do? Maybe we can add a very limited amount of synchronization to Configuration. Looking at a recent version of Configuration.java , it seems that the private updatingResource HashMap and finalParameters HashSet fields the only non-thread-safe collections in Configuration (Java's Properties class is thread-safe). My hunch is that the updatingResource HashMap was the map referred to by the stacktrace posted in this issue. We might be able to use reflection to find this field and inject a synchronized HashMap instead.
          Hide
          aash Andrew Ash added a comment -

          Excellent research Josh!

          I agree that we should pass for now on the driver-side thread-safety issues. All the issues I've encountered so far have been in multiple accesses on the executor side, which the cloning on access approach seems to take care of.

          Show
          aash Andrew Ash added a comment - Excellent research Josh! I agree that we should pass for now on the driver-side thread-safety issues. All the issues I've encountered so far have been in multiple accesses on the executor side, which the cloning on access approach seems to take care of.
          Hide
          apachespark Apache Spark added a comment -

          User 'JoshRosen' has created a pull request for this issue:
          https://github.com/apache/spark/pull/2684

          Show
          apachespark Apache Spark added a comment - User 'JoshRosen' has created a pull request for this issue: https://github.com/apache/spark/pull/2684
          Hide
          aash Andrew Ash added a comment -

          We tested Josh's patch, confirming the fix and measuring the perf regression at ~8%

          Show
          aash Andrew Ash added a comment - We tested Josh's patch, confirming the fix and measuring the perf regression at ~8%
          Hide
          joshrosen Josh Rosen added a comment -

          Issue resolved by pull request 2684
          https://github.com/apache/spark/pull/2684

          Show
          joshrosen Josh Rosen added a comment - Issue resolved by pull request 2684 https://github.com/apache/spark/pull/2684
          Hide
          joshrosen Josh Rosen added a comment -

          I've fixed this in HadoopRDD and applied my fix to all branches. Note that the fix is currently guarded by a configuration option, spark.hadoop.cloneConf. This is in order to avoid unexpected performance regressions when users who were unaffected by this issue choose to upgrade to 1.1.1 or 1.0.3. We'll probably make cloning the default in 1.2.0 and may spend some more time trying to understand its performance implications.

          Note that this does not address the potential for thread-safety issues due to Configuration-sharing on the driver. As described upthread, this is a much harder issue to fix. Since I'm not aware of any cases where this has caused issues on the driver, I'm inclined to wait things out and address that if it's discovered to be an issue.

          I've opened HADOOP-11209 to try to fix the Configuration thread-safety issues upstream, so hopefully this won't be a problem in the future.

          Show
          joshrosen Josh Rosen added a comment - I've fixed this in HadoopRDD and applied my fix to all branches. Note that the fix is currently guarded by a configuration option, spark.hadoop.cloneConf . This is in order to avoid unexpected performance regressions when users who were unaffected by this issue choose to upgrade to 1.1.1 or 1.0.3. We'll probably make cloning the default in 1.2.0 and may spend some more time trying to understand its performance implications. Note that this does not address the potential for thread-safety issues due to Configuration-sharing on the driver. As described upthread, this is a much harder issue to fix. Since I'm not aware of any cases where this has caused issues on the driver, I'm inclined to wait things out and address that if it's discovered to be an issue. I've opened HADOOP-11209 to try to fix the Configuration thread-safety issues upstream, so hopefully this won't be a problem in the future.
          Hide
          ozawa Tsuyoshi Ozawa added a comment -

          Now HADOOP-11209, the problem reported by Josh Rosen, is resolved by Varun Saxena's contribution. Thanks for your reporting.

          Show
          ozawa Tsuyoshi Ozawa added a comment - Now HADOOP-11209 , the problem reported by Josh Rosen , is resolved by Varun Saxena 's contribution. Thanks for your reporting.
          Hide
          ankurmitujjain Ankur Jain added a comment -

          This exists in SPARK 1.4 too...

          Show
          ankurmitujjain Ankur Jain added a comment - This exists in SPARK 1.4 too...
          Hide
          ozawa Tsuyoshi Ozawa added a comment -

          Ankur Jain cc: Josh Rosen the problem is fixed in Hadoop 2.7. Could you build spark with hadoop.version=2.7.1? I'll also backport the patch to 2.6.x, but it takes a bit time to release.

          Show
          ozawa Tsuyoshi Ozawa added a comment - Ankur Jain cc: Josh Rosen the problem is fixed in Hadoop 2.7. Could you build spark with hadoop.version=2.7.1? I'll also backport the patch to 2.6.x, but it takes a bit time to release.
          Hide
          ankurmitujjain Ankur Jain added a comment -

          Thanks Tsuyoshi...
          I thought fix is already done for version 1.0.3, 1.1.1, 1.2.0.
          So 1.4.0 should have this fix with it....

          Anyways this means that on EMR we will face this issue as they are using Hadoop 2.4.0

          Show
          ankurmitujjain Ankur Jain added a comment - Thanks Tsuyoshi... I thought fix is already done for version 1.0.3, 1.1.1, 1.2.0. So 1.4.0 should have this fix with it.... Anyways this means that on EMR we will face this issue as they are using Hadoop 2.4.0
          Hide
          joshrosen Josh Rosen added a comment -

          Ankur Jain, you can try setting spark.hadoop.cloneConf=true in your SparkConf in order to enable additional defensive copying that is designed to guard against this issue. This setting is off by default because this cloning is actually fairly expensive because new Configuration objects are costly to create.

          Show
          joshrosen Josh Rosen added a comment - Ankur Jain , you can try setting spark.hadoop.cloneConf=true in your SparkConf in order to enable additional defensive copying that is designed to guard against this issue. This setting is off by default because this cloning is actually fairly expensive because new Configuration objects are costly to create.

            People

            • Assignee:
              joshrosen Josh Rosen
              Reporter:
              aash Andrew Ash
            • Votes:
              0 Vote for this issue
              Watchers:
              11 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development