Skip to content

[HUDI-9355] Avoid calling FileSystem#close in HoodieHadoopStorage #13243

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

Merged
merged 1 commit into from
May 1, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,8 @@ public boolean createNewFile(StoragePath path) throws IOException {

@Override
public void close() throws IOException {
fs.close();
// Don't close the wrapped `FileSystem` object.
// This will end up closing it for every thread since it
// could be cached across JVM. We don't own that object anyway.
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,11 @@

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.junit.jupiter.api.Test;

import java.io.IOException;

import static org.junit.jupiter.api.Assertions.assertSame;

/**
* Tests {@link HoodieHadoopStorage}.
Expand All @@ -49,4 +54,20 @@ protected Object getConf() {
conf.set(CONF_KEY, CONF_VALUE);
return conf;
}

@Test
void testClose() throws IOException {
Configuration conf = new Configuration();
String path = getTempDir();
FileSystem fileSystem = HadoopFSUtils.getFs(path, conf, true);
HoodieStorage storage = new HoodieHadoopStorage(fileSystem);
storage.close();
// This validates that HoodieHadoopStorage#close does not close the underlying FileSystem
// object. If the underlying FileSystem object is closed, the cache of the object based on
// the path is closed and removed, which causes problems if it is reused elsewhere. Fetching
// the FileSystem object on the same path again in this case returns a different object,
// which can be caught here.
assertSame(fileSystem, storage.getFileSystem());
assertSame(fileSystem, HadoopFSUtils.getFs(getTempDir(), conf, true));
}
}
Loading