[jira] [Updated] (SPARK-2546) Configuration object thread safety issue

2014-10-19 Thread Josh Rosen (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-2546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Josh Rosen updated SPARK-2546:
--
Affects Version/s: 1.2.0
   1.0.2
   1.1.0

 Configuration object thread safety issue
 

 Key: SPARK-2546
 URL: https://issues.apache.org/jira/browse/SPARK-2546
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 0.9.1, 1.0.2, 1.1.0, 1.2.0
Reporter: Andrew Ash
Assignee: Josh Rosen
Priority: Critical
 Fix For: 1.1.1, 1.2.0, 1.0.3


 // 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:
 {quote}
 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.
 {noformat}
 Executor task launch worker-6 daemon prio=10 tid=0x7f91f01fe000 
 nid=0x54b1 runnable [0x7f92d74f1000]
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 
 

[jira] [Updated] (SPARK-2546) Configuration object thread safety issue

2014-08-15 Thread Patrick Wendell (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-2546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Patrick Wendell updated SPARK-2546:
---

Target Version/s: 1.2.0  (was: 1.1.0)

 Configuration object thread safety issue
 

 Key: SPARK-2546
 URL: https://issues.apache.org/jira/browse/SPARK-2546
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 0.9.1
Reporter: Andrew Ash
Assignee: Josh Rosen
Priority: Critical

 // 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:
 {quote}
 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.
 {noformat}
 Executor task launch worker-6 daemon prio=10 tid=0x7f91f01fe000 
 nid=0x54b1 runnable [0x7f92d74f1000]
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 

[jira] [Updated] (SPARK-2546) Configuration object thread safety issue

2014-08-04 Thread Reynold Xin (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-2546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Reynold Xin updated SPARK-2546:
---

Assignee: Josh Rosen

 Configuration object thread safety issue
 

 Key: SPARK-2546
 URL: https://issues.apache.org/jira/browse/SPARK-2546
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 0.9.1
Reporter: Andrew Ash
Assignee: Josh Rosen
Priority: Critical

 // 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:
 {quote}
 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.
 {noformat}
 Executor task launch worker-6 daemon prio=10 tid=0x7f91f01fe000 
 nid=0x54b1 runnable [0x7f92d74f1000]
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 

[jira] [Updated] (SPARK-2546) Configuration object thread safety issue

2014-07-30 Thread Patrick Wendell (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-2546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Patrick Wendell updated SPARK-2546:
---

Priority: Critical  (was: Major)

 Configuration object thread safety issue
 

 Key: SPARK-2546
 URL: https://issues.apache.org/jira/browse/SPARK-2546
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 0.9.1
Reporter: Andrew Ash
Priority: Critical

 // 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:
 {quote}
 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.
 {noformat}
 Executor task launch worker-6 daemon prio=10 tid=0x7f91f01fe000 
 nid=0x54b1 runnable [0x7f92d74f1000]
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 

[jira] [Updated] (SPARK-2546) Configuration object thread safety issue

2014-07-21 Thread Patrick Wendell (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-2546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Patrick Wendell updated SPARK-2546:
---

Target Version/s: 1.1.0

 Configuration object thread safety issue
 

 Key: SPARK-2546
 URL: https://issues.apache.org/jira/browse/SPARK-2546
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 0.9.1
Reporter: Andrew Ash

 // 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:
 {quote}
 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.
 {noformat}
 Executor task launch worker-6 daemon prio=10 tid=0x7f91f01fe000 
 nid=0x54b1 runnable [0x7f92d74f1000]
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