Improved object store memory behavior with respect /dev/shm size (#14629)

This commit is contained in:
Eric Liang 2021-03-11 17:29:06 -08:00 committed by GitHub
parent 5e2a3df7cd
commit ee2bf0f989
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
3 changed files with 18 additions and 9 deletions

View file

@ -1681,14 +1681,17 @@ def determine_plasma_store_config(object_store_memory,
# /dev/shm.
if shm_avail > object_store_memory:
plasma_directory = "/dev/shm"
elif not os.environ.get("RAY_OBJECT_STORE_ALLOW_SLOW_STORAGE"):
elif (not os.environ.get("RAY_OBJECT_STORE_ALLOW_SLOW_STORAGE")
and object_store_memory >
ray_constants.REQUIRE_SHM_SIZE_THRESHOLD):
raise ValueError(
"The configured object store size exceeds the capacity of "
"/dev/shm. This will harm performance. To proceed "
"regardless of this warning, you can set "
"RAY_OBJECT_STORE_ALLOW_SLOW_STORAGE=1. Consider deleting "
"files in /dev/shm or increasing its size with "
"--shm-size in Docker.")
"The configured object store size ({} GB) exceeds "
"/dev/shm size ({} GB). This will harm performance. "
"Consider deleting files in /dev/shm or increasing its "
"size with "
"--shm-size in Docker. To ignore this warning, "
"set RAY_OBJECT_STORE_ALLOW_SLOW_STORAGE=1.".format(
object_store_memory / 1e9, shm_avail / 1e9))
else:
plasma_directory = ray._private.utils.get_user_temp_dir()
logger.warning(

View file

@ -34,6 +34,9 @@ OBJECT_STORE_MINIMUM_MEMORY_BYTES = 75 * 1024 * 1024
DEFAULT_REDIS_MAX_MEMORY_BYTES = 10**10
# The smallest cap on the memory used by Redis that we allow.
REDIS_MINIMUM_MEMORY_BYTES = 10**7
# Above this number of bytes, raise an error by default unless the user sets
# RAY_ALLOW_SLOW_STORAGE=1. This avoids swapping with large object stores.
REQUIRE_SHM_SIZE_THRESHOLD = 10**10
# If a user does not specify a port for the primary Ray service,
# we attempt to start the service running at this port.
DEFAULT_PORT = 6379

View file

@ -167,10 +167,13 @@ class ResourceSpec(
avail_memory *
ray_constants.DEFAULT_OBJECT_STORE_MEMORY_PROPORTION)
max_cap = ray_constants.DEFAULT_OBJECT_STORE_MAX_MEMORY_BYTES
# Cap by shm size by default to avoid low performance.
# Cap by shm size by default to avoid low performance, but don't
# go lower than REQUIRE_SHM_SIZE_THRESHOLD.
if sys.platform == "linux" or sys.platform == "linux2":
shm_avail = ray._private.utils.get_shared_memory_bytes()
max_cap = min(shm_avail, max_cap)
max_cap = min(
max(ray_constants.REQUIRE_SHM_SIZE_THRESHOLD, shm_avail),
max_cap)
# Cap memory to avoid memory waste and perf issues on large nodes
if object_store_memory > max_cap:
logger.debug(