1
0

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:
Vinoth Chandar
2019-03-27 18:19:41 -07:00
committed by vinoth chandar
parent f1410bfdcd
commit 5847f0c934
2 changed files with 12 additions and 6 deletions

View File

@@ -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