-
Notifications
You must be signed in to change notification settings - Fork 2.4k
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
[HUDI-4526] Improve spillableMapBasePath when disk directory is full #6284
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -204,4 +204,40 @@ public static Option<byte[]> readDataFromPath(FileSystem fileSystem, org.apache. | |||
public static Option<byte[]> readDataFromPath(FileSystem fileSystem, org.apache.hadoop.fs.Path detailPath) { | ||||
return readDataFromPath(fileSystem, detailPath, false); | ||||
} | ||||
|
||||
/** | ||||
* Return the configured local directories where hudi can write files. This | ||||
* method does not create any directories on its own, it only encapsulates the | ||||
* logic of locating the local directories according to deployment mode. | ||||
*/ | ||||
public static String[] getConfiguredLocalDirs() { | ||||
if (isRunningInYarnContainer()) { | ||||
// If we are in yarn mode, systems can have different disk layouts so we must set it | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We totally ignore the option through hoodie write config then ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
First, get the local dir from the yarn container. If you can't get it, you can get it from the hoodie write config. In this way, users on yarn basically do not need to actively modify the hoodie write config. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I didn't see the logic. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
hudi/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java Line 98 in 0946f43
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks, should we take the write config as higher priority then ? If user configured it explicitly. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
It may be necessary to remove the default value
|
||||
// to what Yarn on this system said was available. Note this assumes that Yarn has | ||||
// created the directories already, and that they are secured so that only the | ||||
// user has access to them. | ||||
return getYarnLocalDirs().split(","); | ||||
} else if (System.getProperty("java.io.tmpdir") != null) { | ||||
return System.getProperty("java.io.tmpdir").split(","); | ||||
} else { | ||||
return null; | ||||
} | ||||
} | ||||
|
||||
private static boolean isRunningInYarnContainer() { | ||||
// These environment variables are set by YARN. | ||||
return System.getenv("CONTAINER_ID") != null; | ||||
} | ||||
|
||||
/** | ||||
* Get the Yarn approved local directories. | ||||
*/ | ||||
private static String getYarnLocalDirs() { | ||||
String localDirs = Option.of(System.getenv("LOCAL_DIRS")).orElse(""); | ||||
|
||||
if (localDirs.isEmpty()) { | ||||
throw new HoodieIOException("Yarn Local dirs can't be empty"); | ||||
} | ||||
return localDirs; | ||||
} | ||||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So when we clean these files then ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
BitCaskDiskMap
setwriteOnlyFile.deleteOnExit()
RocksDbDiskMap::close
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So why the dir is full if it is cleaned in time ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When spark writes multiple jobs concurrently, there are not only hudi jobs, but the
tmp
directory shared by many jobs will cause it to explode.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
not sure if we can do this. spillableMapbase path is configurable. If one does not want "/tmp/" which is the default, they can always override using the configs.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is more troublesome for users to use, and this option needs to be specified additionally, and considering that the
basepath
will definitely be in a large data disk, the temporary directory can be put into the same disk as thebasepath
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
basepath is hdfs path, spillableMapbase_path is local path,
It is wrong to use hdfs path directly as spillableMapbase_path