Fix HUDI-27 : Support num_cores > 1 for writing through spark
- Users using spark.executor.cores > 1 used to fail due to "FileSystem closed" - This is due to HoodieWrapperFileSystem closing the wrapped filesytem obj - FileSystem.getInternal caching code races threads and closes the extra fs instance(s) - Bumped up num cores in tests to 8, speeds up tests by 3-4 mins
This commit is contained in:
committed by
vinoth chandar
parent
f1410bfdcd
commit
5847f0c934
@@ -536,8 +536,9 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
fileSystem.close();
|
||||
public void close() {
|
||||
// Don't close the underlying `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.
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
Reference in New Issue
Block a user