Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Accessing through the Hadoop FileContext API will cause client-side memory leaks. #18479

Open
qian0817 opened this issue Jan 2, 2024 · 0 comments
Labels
type-bug This issue is about a bug

Comments

@qian0817
Copy link
Contributor

qian0817 commented Jan 2, 2024

Alluxio Version:
2.x

Describe the bug
We found Full GC and OOM behavior on YARN Nodemanger in an environment. after analyzing the heap dump, we found that the ConfigHashSync of the alluxio client and related classes were not being garbage collected properly.
image
image

We believe this is due to the client not being properly closed, so we modified the BaseFileSystem class in the client code to see where the client was not properly closed.

  public BaseFileSystem(FileSystemContext fsContext) {
    // ...
    LOG.info("constructing BaseFileSystem with context id {}", mFsContext.getId(), new Exception("print stack trace"));
  }
  public synchronized void close() throws IOException {
      // ...
    LOG.info("closing BaseFileSystem with context id {}", mFsContext.getId(), new Exception("print stack trace"));
  }

In the end, we get the following log:

2024-01-02 11:17:19,817 [2146392] - INFO  [LogAggregationService #56:BaseFileSystem@109] - constructing BaseFileSystem with context id app-2020615593602897283
java.lang.Exception: print stack trace
	at alluxio.client.file.BaseFileSystem.<init>(BaseFileSystem.java:109)
	at alluxio.client.file.FileSystem$Factory.create(FileSystem.java:146)
	at alluxio.client.file.FileSystem$Factory.create(FileSystem.java:127)
	at alluxio.hadoop.AbstractFileSystem.initialize(AbstractFileSystem.java:533)
	at alluxio.hadoop.AbstractFileSystem.initialize(AbstractFileSystem.java:472)
	at org.apache.hadoop.fs.DelegateToFileSystem.<init>(DelegateToFileSystem.java:52)
	at alluxio.hadoop.AlluxioFileSystem.<init>(AlluxioFileSystem.java:50)
	at sun.reflect.GeneratedConstructorAccessor64.newInstance(Unknown Source)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at org.apache.hadoop.fs.AbstractFileSystem.newInstance(AbstractFileSystem.java:135)
	at org.apache.hadoop.fs.AbstractFileSystem.createFileSystem(AbstractFileSystem.java:173)
	at org.apache.hadoop.fs.AbstractFileSystem.get(AbstractFileSystem.java:258)
	at org.apache.hadoop.fs.FileContext$2.run(FileContext.java:342)
	at org.apache.hadoop.fs.FileContext$2.run(FileContext.java:339)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1893)
	at org.apache.hadoop.fs.FileContext.getAbstractFileSystem(FileContext.java:339)
	at org.apache.hadoop.fs.FileContext.getFileContext(FileContext.java:465)
	at org.apache.hadoop.fs.FileContext.getFileContext(FileContext.java:491)
	at org.apache.hadoop.fs.FileContext.getFileContext(FileContext.java:461)
	at org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat$LogWriter$1.run(AggregatedLogFormat.java:476)
	at org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat$LogWriter$1.run(AggregatedLogFormat.java:473)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1893)
	at org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat$LogWriter.initialize(AggregatedLogFormat.java:472)
	at org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController.initializeWriter(LogAggregationTFileController.java:90)
	at org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AppLogAggregatorImpl.uploadLogsForContainers(AppLogAggregatorImpl.java:306)
	at org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AppLogAggregatorImpl.doAppLogAggregation(AppLogAggregatorImpl.java:459)
	at org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AppLogAggregatorImpl.run(AppLogAggregatorImpl.java:415)
	at org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService$1.run(LogAggregationService.java:265)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

When the method FileContext.getFileContext is called, it creates an instance of alluxio.hadoop.AlluxioFileSystem to access Alluxio. However, the base class org.apache.hadoop.fs.AbstractFileSystem does not have a close method to release the held resources, which can lead to memory leaks.

To Reproduce

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;

import java.net.URI;

public class Main {
    public static void main(String[] args) throws Exception {
        while (true){
            Configuration conf = new Configuration();
            conf.set("fs.AbstractFileSystem.alluxio.impl", "alluxio.hadoop.AlluxioFileSystem");
            // every call will create new filesystem but not close
            FileContext fileContext = FileContext.getFileContext(URI.create("alluxio://localhost:19998/"), conf);
            FileStatus fileStatus = fileContext.getFileStatus(new Path("/"));
            System.out.println(fileStatus);
            Thread.sleep(200);
        }
    }
}

Expected behavior
Will not produce memory leaks.

Urgency
High.

Are you planning to fix it
yes.

Additional context

@qian0817 qian0817 added the type-bug This issue is about a bug label Jan 2, 2024
alluxio-bot pushed a commit that referenced this issue Mar 6, 2024
### What changes are proposed in this pull request?

Fix #18479

### Why are the changes needed?

Fix the memory leak issue when the client accesses Alluxio through Hadoop FileContext API.

### Does this PR introduce any user facing changes?

No.
			pr-link: #18480
			change-id: cid-f36ec718ebcbc61fdee79cef2ecc8731d0ba2ee5
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type-bug This issue is about a bug
Projects
None yet
Development

No branches or pull requests

1 participant