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

Gopal V updated HADOOP-16461:
-----------------------------
    Description: 
{code}
      fs = createFileSystem(uri, conf);
      synchronized (this) { // refetch the lock again
        FileSystem oldfs = map.get(key);
        if (oldfs != null) { // a file system is created while lock is releasing
          fs.close(); // close the new file system
          return oldfs;  // return the old file system
        }

        // now insert the new file system into the map
        if (map.isEmpty()
                && !ShutdownHookManager.get().isShutdownInProgress()) {
          ShutdownHookManager.get().addShutdownHook(clientFinalizer, 
SHUTDOWN_HOOK_PRIORITY);
        }
        fs.key = key;
        map.put(key, fs);
        if (conf.getBoolean(
            FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
          toAutoClose.add(key);
        }
        return fs;
      }
{code}

The lock now has a ShutdownHook creation, which ends up doing 

{code}
    HookEntry(Runnable hook, int priority) {
      this(hook, priority,
          getShutdownTimeout(new Configuration()),
          TIME_UNIT_DEFAULT);
    }
{code}

which ends up doing a "new Configuration()" within the locked section.

This indirectly hurts the cache hit scenarios as well, since if the lock on 
this is held, then the other section cannot be entered either.

{code}
I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms
org.apache.hadoop.fs.FileSystem$Cache.getInternal(URI, Configuration, 
FileSystem$Cache$Key) FileSystem.java:3345
org.apache.hadoop.fs.FileSystem$Cache.get(URI, Configuration) 
FileSystem.java:3320
org.apache.hadoop.fs.FileSystem.get(URI, Configuration) FileSystem.java:479
org.apache.hadoop.fs.FileSystem.getLocal(Configuration) FileSystem.java:435
{code}

slowing down the RawLocalFileSystem when there are other threads creating HDFS 
FileSystem objects at the same time.

  was:
{code}
      fs = createFileSystem(uri, conf);
      synchronized (this) { // refetch the lock again
        FileSystem oldfs = map.get(key);
        if (oldfs != null) { // a file system is created while lock is releasing
          fs.close(); // close the new file system
          return oldfs;  // return the old file system
        }

        // now insert the new file system into the map
        if (map.isEmpty()
                && !ShutdownHookManager.get().isShutdownInProgress()) {
          ShutdownHookManager.get().addShutdownHook(clientFinalizer, 
SHUTDOWN_HOOK_PRIORITY);
        }
        fs.key = key;
        map.put(key, fs);
        if (conf.getBoolean(
            FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
          toAutoClose.add(key);
        }
        return fs;
      }
{code}

The lock now has a ShutdownHook creation, which ends up doing 

{code}
    HookEntry(Runnable hook, int priority) {
      this(hook, priority,
          getShutdownTimeout(new Configuration()),
          TIME_UNIT_DEFAULT);
    }
{code}

which ends up doing a "new Configuration()" within the locked section.




> Regression: FileSystem cache lock parses XML within the lock
> ------------------------------------------------------------
>
>                 Key: HADOOP-16461
>                 URL: https://issues.apache.org/jira/browse/HADOOP-16461
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: filecache
>            Reporter: Gopal V
>            Priority: Major
>
> {code}
>       fs = createFileSystem(uri, conf);
>       synchronized (this) { // refetch the lock again
>         FileSystem oldfs = map.get(key);
>         if (oldfs != null) { // a file system is created while lock is 
> releasing
>           fs.close(); // close the new file system
>           return oldfs;  // return the old file system
>         }
>         // now insert the new file system into the map
>         if (map.isEmpty()
>                 && !ShutdownHookManager.get().isShutdownInProgress()) {
>           ShutdownHookManager.get().addShutdownHook(clientFinalizer, 
> SHUTDOWN_HOOK_PRIORITY);
>         }
>         fs.key = key;
>         map.put(key, fs);
>         if (conf.getBoolean(
>             FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
>           toAutoClose.add(key);
>         }
>         return fs;
>       }
> {code}
> The lock now has a ShutdownHook creation, which ends up doing 
> {code}
>     HookEntry(Runnable hook, int priority) {
>       this(hook, priority,
>           getShutdownTimeout(new Configuration()),
>           TIME_UNIT_DEFAULT);
>     }
> {code}
> which ends up doing a "new Configuration()" within the locked section.
> This indirectly hurts the cache hit scenarios as well, since if the lock on 
> this is held, then the other section cannot be entered either.
> {code}
> I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(URI, Configuration, 
> FileSystem$Cache$Key) FileSystem.java:3345
> org.apache.hadoop.fs.FileSystem$Cache.get(URI, Configuration) 
> FileSystem.java:3320
> org.apache.hadoop.fs.FileSystem.get(URI, Configuration) FileSystem.java:479
> org.apache.hadoop.fs.FileSystem.getLocal(Configuration) FileSystem.java:435
> {code}
> slowing down the RawLocalFileSystem when there are other threads creating 
> HDFS FileSystem objects at the same time.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to