Skip to content

Commit

Permalink
[Fix](hdfs-fs)The cache expiration should explicitly release the held fs
Browse files Browse the repository at this point in the history
  - Implement a removal listener to log and close file systems upon removal.
  - Safely handle IOException during file system closure.
- Improved logging for file system closure and handling of potential exceptions.
  • Loading branch information
CalvinKirs committed Aug 1, 2024
1 parent 4c22f2c commit d9e3536
Show file tree
Hide file tree
Showing 4 changed files with 58 additions and 6 deletions.
30 changes: 25 additions & 5 deletions fe/fe-core/src/main/java/org/apache/doris/fs/FileSystemCache.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,19 +19,26 @@

import org.apache.doris.common.CacheFactory;
import org.apache.doris.common.Config;
import org.apache.doris.common.CustomThreadFactory;
import org.apache.doris.common.Pair;
import org.apache.doris.fs.remote.RemoteFileSystem;

import com.github.benmanes.caffeine.cache.LoadingCache;
import org.apache.hadoop.conf.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.OptionalLong;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;

public class FileSystemCache {

private static final Logger LOG = LoggerFactory.getLogger(FileSystemCache.class);
private final LoadingCache<FileSystemCacheKey, RemoteFileSystem> fileSystemCache;

public FileSystemCache() {
Expand All @@ -42,7 +49,20 @@ public FileSystemCache() {
Config.max_remote_file_system_cache_num,
false,
null);
fileSystemCache = fsCacheFactory.buildCache(this::loadFileSystem);
CustomThreadFactory threadFactory = new CustomThreadFactory("fs-cache-thread");
ExecutorService executor = Executors.newSingleThreadExecutor(threadFactory);
fileSystemCache = fsCacheFactory.buildCache(this::loadFileSystem, (key, fs, removalCause) -> {
if (key != null) {
LOG.info("Close file system: {}", key.fsIdent);
}
try {
if (fs != null) {
fs.close();
}
} catch (IOException e) {
LOG.warn("Failed to close file system", e);
}
}, executor);
}

private RemoteFileSystem loadFileSystem(FileSystemCacheKey key) {
Expand All @@ -63,9 +83,9 @@ public static class FileSystemCacheKey {
private final Configuration conf;

public FileSystemCacheKey(Pair<FileSystemType, String> fs,
Map<String, String> properties,
String bindBrokerName,
Configuration conf) {
Map<String, String> properties,
String bindBrokerName,
Configuration conf) {
this.type = fs.first;
this.fsIdent = fs.second;
this.properties = properties;
Expand All @@ -74,7 +94,7 @@ public FileSystemCacheKey(Pair<FileSystemType, String> fs,
}

public FileSystemCacheKey(Pair<FileSystemType, String> fs,
Map<String, String> properties, String bindBrokerName) {
Map<String, String> properties, String bindBrokerName) {
this(fs, properties, bindBrokerName, null);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,15 +30,20 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;

import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantLock;

public abstract class RemoteFileSystem extends PersistentFileSystem {
public abstract class RemoteFileSystem extends PersistentFileSystem implements Closeable {
// this field will be visited by multi-threads, better use volatile qualifier
protected volatile org.apache.hadoop.fs.FileSystem dfsFileSystem = null;
private final ReentrantLock fsLock = new ReentrantLock();
protected static final AtomicBoolean closed = new AtomicBoolean(false);

public RemoteFileSystem(String name, StorageBackend.StorageType type) {
super(name, type);
Expand Down Expand Up @@ -120,4 +125,18 @@ public Status renameDir(String origFilePath,

return rename(origFilePath, destFilePath);
}

@Override
public void close() throws IOException {
fsLock.lock();
try {
if (!closed.getAndSet(true)) {
if (dfsFileSystem != null) {
dfsFileSystem.close();
}
}
} finally {
fsLock.unlock();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,8 +58,15 @@ private void initFsProperties() {

@Override
protected FileSystem nativeFileSystem(String remotePath) throws UserException {
//todo Extracting a common method to achieve logic reuse
if (closed.get()) {
throw new UserException("FileSystem is closed.");
}
if (dfsFileSystem == null) {
synchronized (this) {
if (closed.get()) {
throw new UserException("FileSystem is closed.");
}
if (dfsFileSystem == null) {
Configuration conf = DFSFileSystem.getHdfsConf(ifNotSetFallbackToSimpleAuth());
System.setProperty("com.amazonaws.services.s3.enableV4", "true");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,8 +75,14 @@ public DFSFileSystem(StorageBackend.StorageType type, Map<String, String> proper
@VisibleForTesting
@Override
public FileSystem nativeFileSystem(String remotePath) throws UserException {
if (closed.get()) {
throw new UserException("FileSystem is closed.");
}
if (dfsFileSystem == null) {
synchronized (this) {
if (closed.get()) {
throw new UserException("FileSystem is closed.");
}
if (dfsFileSystem == null) {
Configuration conf = getHdfsConf(ifNotSetFallbackToSimpleAuth());
for (Map.Entry<String, String> propEntry : properties.entrySet()) {
Expand Down

0 comments on commit d9e3536

Please sign in to comment.