You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Steve Loughran (Jira)" <ji...@apache.org> on 2020/10/19 15:11:02 UTC

[jira] [Commented] (HADOOP-17313) FileSystem.get to support slow-to-instantiate FS clients

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

Steve Loughran commented on HADOOP-17313:
-----------------------------------------

Today (roughly)
{code}
private FileSystem getInternal(URI uri, Configuration conf, Key key)
    throws IOException{
  FileSystem fs;
  synchronized (this) {
    fs = map.get(key);
  }
  if (fs != null) {
    return fs;
  }

  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
    }
    fs.key = key;
    map.put(key, fs);
    return fs;
  }
{code}

{code}
// per-cache (i.e. per UGI) lock on FS creation.
private final Object creatorLock = new Object();

private FileSystem getInternal(URI uri, Configuration conf, Key key)
    throws IOException{
  FileSystem fs;
  synchronized (this) {
    fs = map.get(key);
  }
  if (fs != null) {
    return fs;
  }
  // fs not yet created, acquire lock
  // to construct an instance.
  synchronized(this.creatorLock) {
    // repeat inside the sync block
    synchronized (this) {
      fs = map.get(key);
    }
    if (fs != null) {
      return fs;
    }
    // fs still doesn't exist, create
   fs = createFileSystem(uri, conf);
   
   synchronized (this) { // refetch the lock again
     // this thread is guaranteed to be the sole one adding objects, so no
     // need to worry about lost objects.
    fs.key = key;
    map.put(key, fs);
    return fs;
  }

}
{code}

once thread T1 starts to create a store connector, all other threads block until it is in, one thread Tx will acquire the create lock, see that there's an entry in the map, return it, ... same for all the others. Only one instance will be created.

There's a cost: if two threads T1 and T2 want to create connectors for the same user for two different stores, T2 will block until T1 is complete. Question is: how often does that occur, and, given we are seeing locking problems internally, do we care that much? 

Maybe: make it a semaphore, where #of active entries is limited. If you are happier with wastage, make a higher value; if you want minimimal parallelism, set to one.




> FileSystem.get to support slow-to-instantiate FS clients
> --------------------------------------------------------
>
>                 Key: HADOOP-17313
>                 URL: https://issues.apache.org/jira/browse/HADOOP-17313
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs, fs/azure, fs/s3
>    Affects Versions: 3.3.0
>            Reporter: Steve Loughran
>            Priority: Major
>
> A recurrent problem in processes with many worker threads (hive, spark etc) is that calling `FileSystem.get(URI-to-object-store)` triggers the creation and then discard of many FS clients -all but one for the same URL. As well as the direct performance hit, this can exacerbate locking problems and make instantiation a lot slower than it would otherwise be.
> This has been observed with the S3A and ABFS connectors.
> The ultimate solution here would probably be something more complicated to ensure that only one thread was ever creating a connector for a given URL -the rest would wait for it to be initialized. This would (a) reduce contention & CPU, IO network load, and (b) reduce the time for all but the first thread to resume processing to that of the remaining time in .initialize(). This would also benefit the S3A connector.
> We'd need something like
> # A (per-user) map of filesystems being created <URI, FileSystem>
> # split createFileSystem into two: instantiateFileSystem and initializeFileSystem
> # each thread to instantiate the FS, put() it into the new map
> # If there was one already, discard the old one and wait for the new one to be ready via a call to Object.wait()
> # If there wasn't an entry, call initializeFileSystem) and then, finally, call Object.notifyAll(), and move it from the map of filesystems being initialized to the map of created filesystems
> This sounds too straightforward to be that simple; the troublespots are probably related to race conditions moving entries between the two maps and making sure that no thread will block on the FS being initialized while it has already been initialized (and so wait() will block forever).
> Rather than seek perfection, it may be safest go for a best-effort optimisation of the #of FS instances created/initialized. That is: its better to maybe create a few more FS instances than needed than it is to block forever.
> Something is doable here, it's just not quick-and-dirty. Testing will be "fun"; probably best to isolate this new logic somewhere where we can simulate slow starts on one thread with many other threads waiting for it.
> A simpler option would be to have a lock on the construction process: only one FS can be instantiated per user at a a time.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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