Ship plasma store with Ray (#7901)

This commit is contained in:
Siyuan (Ryans) Zhuang 2020-06-03 17:44:34 -07:00 committed by GitHub
parent a24d117c68
commit ea05ebe89e
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
64 changed files with 17430 additions and 748 deletions

View file

@ -243,6 +243,202 @@ cc_library(
# === End of rpc definitions === # === End of rpc definitions ===
# === Begin of plasma definitions ===
# TODO(mehrdadn): (How to) support dynamic linking?
PROPAGATED_WINDOWS_DEFINES = ["ARROW_STATIC"]
PLASMA_COPTS = COPTS + select({
"@bazel_tools//src/conditions:windows": [
"-D" + "WIN32_REPLACE_FD_APIS",
"/FI" + "win32fd.h",
] + ["-D" + define for define in PROPAGATED_WINDOWS_DEFINES],
"//conditions:default": [
"-DARROW_USE_GLOG",
],
})
PLASMA_LINKOPTS = [] + select({
"@bazel_tools//src/conditions:windows": [
"-DefaultLib:" + "ws2_32.lib",
],
"//conditions:default": [
],
})
cc_library(
name = "plasma_client",
srcs = [
"src/ray/plasma/client.cc",
"src/ray/plasma/common.cc",
"src/ray/plasma/fling.cc",
"src/ray/plasma/io.cc",
"src/ray/plasma/malloc.cc",
"src/ray/plasma/plasma.cc",
"src/ray/plasma/protocol.cc",
],
hdrs = [
"src/ray/plasma/client.h",
"src/ray/plasma/common.h",
"src/ray/plasma/common_generated.h",
"src/ray/plasma/compat.h",
"src/ray/plasma/external_store.h",
"src/ray/plasma/fling.h",
"src/ray/plasma/io.h",
"src/ray/plasma/malloc.h",
"src/ray/plasma/plasma.h",
"src/ray/plasma/plasma_generated.h",
"src/ray/plasma/protocol.h",
],
copts = PLASMA_COPTS,
defines = select({
"@bazel_tools//src/conditions:windows": PROPAGATED_WINDOWS_DEFINES,
"//conditions:default": [],
}),
linkopts = PLASMA_LINKOPTS,
strip_include_prefix = "src/ray",
deps = [
":common_fbs",
":plasma_fbs",
":platform_shims",
"@arrow",
"@com_github_google_glog//:glog",
],
)
cc_binary(
name = "libplasma_java.so",
srcs = [
"src/ray/plasma/lib/java/org_apache_arrow_plasma_PlasmaClientJNI.cc",
"src/ray/plasma/lib/java/org_apache_arrow_plasma_PlasmaClientJNI.h",
":jni.h",
":jni_md.h",
],
copts = PLASMA_COPTS,
includes = [
"src/ray",
],
linkshared = 1,
linkstatic = 1,
deps = [":plasma_client"],
)
genrule(
name = "copy_jni_h",
srcs = ["@bazel_tools//tools/jdk:jni_header"],
outs = ["jni.h"],
cmd = "cp -f $< $@",
)
genrule(
name = "copy_jni_md_h",
srcs = select({
"@bazel_tools//src/conditions:windows": ["@bazel_tools//tools/jdk:jni_md_header-windows"],
"@bazel_tools//src/conditions:darwin": ["@bazel_tools//tools/jdk:jni_md_header-darwin"],
"//conditions:default": ["@bazel_tools//tools/jdk:jni_md_header-linux"],
}),
outs = ["jni_md.h"],
cmd = "cp -f $< $@",
)
genrule(
name = "plasma-jni-darwin-compat",
srcs = [":libplasma_java.so"],
outs = ["libplasma_java.dylib"],
cmd = "cp $< $@",
output_to_bindir = 1,
)
cc_library(
name = "ae",
srcs = [
"src/ray/plasma/thirdparty/ae/ae.c",
],
hdrs = [
"src/ray/plasma/thirdparty/ae/ae.h",
"src/ray/plasma/thirdparty/ae/ae_epoll.c",
"src/ray/plasma/thirdparty/ae/ae_evport.c",
"src/ray/plasma/thirdparty/ae/ae_kqueue.c",
"src/ray/plasma/thirdparty/ae/ae_select.c",
"src/ray/plasma/thirdparty/ae/config.h",
"src/ray/plasma/thirdparty/ae/zmalloc.h",
],
copts = PLASMA_COPTS,
includes = [
"src/ray/plasma/thirdparty/ae",
],
strip_include_prefix = "src/ray",
deps = [
":platform_shims",
],
)
cc_library(
name = "plasma_lib",
srcs = [
"src/ray/plasma/dlmalloc.cc",
"src/ray/plasma/events.cc",
"src/ray/plasma/eviction_policy.cc",
"src/ray/plasma/external_store.cc",
"src/ray/plasma/plasma_allocator.cc",
"src/ray/plasma/quota_aware_policy.cc",
],
hdrs = [
"src/ray/plasma/events.h",
"src/ray/plasma/eviction_policy.h",
"src/ray/plasma/external_store.h",
"src/ray/plasma/plasma_allocator.h",
"src/ray/plasma/quota_aware_policy.h",
"src/ray/plasma/store.h",
"src/ray/plasma/thirdparty/dlmalloc.c",
],
copts = PLASMA_COPTS,
linkopts = PLASMA_LINKOPTS,
strip_include_prefix = "src/ray",
deps = [
":ae",
":plasma_client",
":platform_shims",
"@com_github_google_glog//:glog",
],
)
cc_binary(
name = "plasma_store_server",
srcs = [
"src/ray/plasma/store.cc",
],
copts = PLASMA_COPTS,
visibility = ["//visibility:public"],
deps = [
":plasma_lib",
":platform_shims",
],
)
FLATC_ARGS = [
"--gen-object-api",
"--gen-mutable",
"--scoped-enums",
]
flatbuffer_cc_library(
name = "common_fbs",
srcs = ["src/ray/plasma/common.fbs"],
flatc_args = FLATC_ARGS,
out_prefix = "src/ray/plasma/",
)
flatbuffer_cc_library(
name = "plasma_fbs",
srcs = ["src/ray/plasma/plasma.fbs"],
flatc_args = FLATC_ARGS,
includes = ["src/ray/plasma/common.fbs"],
out_prefix = "src/ray/plasma/",
)
# === End of plasma definitions ===
cc_library( cc_library(
name = "ray_common", name = "ray_common",
srcs = glob( srcs = glob(
@ -267,6 +463,7 @@ cc_library(
":common_cc_proto", ":common_cc_proto",
":gcs_cc_proto", ":gcs_cc_proto",
":node_manager_fbs", ":node_manager_fbs",
":plasma_client",
":ray_util", ":ray_util",
"@boost//:asio", "@boost//:asio",
"@com_github_grpc_grpc//:grpc++", "@com_github_grpc_grpc//:grpc++",
@ -275,7 +472,6 @@ cc_library(
"@com_google_absl//absl/memory", "@com_google_absl//absl/memory",
"@com_google_googletest//:gtest", "@com_google_googletest//:gtest",
"@msgpack", "@msgpack",
"@plasma//:plasma_client",
], ],
) )
@ -453,6 +649,7 @@ cc_library(
":node_manager_fbs", ":node_manager_fbs",
":node_manager_rpc", ":node_manager_rpc",
":object_manager", ":object_manager",
":plasma_client",
":ray_common", ":ray_common",
":ray_util", ":ray_util",
":service_based_gcs_client_lib", ":service_based_gcs_client_lib",
@ -468,7 +665,6 @@ cc_library(
"@io_opencensus_cpp//opencensus/exporters/stats/prometheus:prometheus_exporter", "@io_opencensus_cpp//opencensus/exporters/stats/prometheus:prometheus_exporter",
"@io_opencensus_cpp//opencensus/stats", "@io_opencensus_cpp//opencensus/stats",
"@io_opencensus_cpp//opencensus/tags", "@io_opencensus_cpp//opencensus/tags",
"@plasma//:plasma_client",
], ],
) )
@ -533,17 +729,17 @@ cc_binary(
cc_test( cc_test(
name = "core_worker_test", name = "core_worker_test",
srcs = ["src/ray/core_worker/test/core_worker_test.cc"], srcs = ["src/ray/core_worker/test/core_worker_test.cc"],
args = ["$(location @plasma//:plasma_store_server) $(location raylet) $(location raylet_monitor) $(location mock_worker) $(location gcs_server) $(location redis-cli) $(location redis-server) $(location libray_redis_module.so)"], args = ["$(location //:plasma_store_server) $(location raylet) $(location raylet_monitor) $(location mock_worker) $(location gcs_server) $(location redis-cli) $(location redis-server) $(location libray_redis_module.so)"],
copts = COPTS, copts = COPTS,
data = [ data = [
"//:gcs_server", "//:gcs_server",
"//:libray_redis_module.so", "//:libray_redis_module.so",
"//:mock_worker", "//:mock_worker",
"//:plasma_store_server",
"//:raylet", "//:raylet",
"//:raylet_monitor", "//:raylet_monitor",
"//:redis-cli", "//:redis-cli",
"//:redis-server", "//:redis-server",
"@plasma//:plasma_store_server",
], ],
deps = [ deps = [
":core_worker_lib", ":core_worker_lib",
@ -1002,10 +1198,10 @@ cc_library(
":gcs", ":gcs",
":object_manager_fbs", ":object_manager_fbs",
":object_manager_rpc", ":object_manager_rpc",
":plasma_client",
":ray_common", ":ray_common",
":ray_util", ":ray_util",
"@boost//:asio", "@boost//:asio",
"@plasma//:plasma_client",
], ],
) )
@ -1076,13 +1272,13 @@ cc_library(
], ],
visibility = ["//visibility:public"], visibility = ["//visibility:public"],
deps = [ deps = [
":plasma_client",
":sha256", ":sha256",
"@boost//:asio", "@boost//:asio",
"@com_github_google_glog//:glog", "@com_github_google_glog//:glog",
"@com_google_absl//absl/synchronization", "@com_google_absl//absl/synchronization",
"@com_google_absl//absl/time", "@com_google_absl//absl/time",
"@com_google_googletest//:gtest_main", "@com_google_googletest//:gtest_main",
"@plasma//:plasma_client",
], ],
) )
@ -1375,19 +1571,6 @@ cc_test(
], ],
) )
FLATC_ARGS = [
"--gen-object-api",
"--gen-mutable",
"--scoped-enums",
]
flatbuffer_cc_library(
name = "common_fbs",
srcs = ["@plasma//:cpp/src/plasma/common.fbs"],
flatc_args = FLATC_ARGS,
out_prefix = "src/ray/common/",
)
flatbuffer_cc_library( flatbuffer_cc_library(
name = "node_manager_fbs", name = "node_manager_fbs",
srcs = ["src/ray/raylet/format/node_manager.fbs"], srcs = ["src/ray/raylet/format/node_manager.fbs"],
@ -1471,7 +1654,10 @@ cc_binary(
"@bazel_tools//src/conditions:windows": ["@bazel_tools//tools/jdk:jni_md_header-windows"], "@bazel_tools//src/conditions:windows": ["@bazel_tools//tools/jdk:jni_md_header-windows"],
"@bazel_tools//src/conditions:darwin": ["@bazel_tools//tools/jdk:jni_md_header-darwin"], "@bazel_tools//src/conditions:darwin": ["@bazel_tools//tools/jdk:jni_md_header-darwin"],
"//conditions:default": ["@bazel_tools//tools/jdk:jni_md_header-linux"], "//conditions:default": ["@bazel_tools//tools/jdk:jni_md_header-linux"],
}), }) + [
":jni.h",
":jni_md.h",
],
copts = COPTS, copts = COPTS,
includes = [ includes = [
"src", "src",
@ -1639,7 +1825,7 @@ genrule(
"//:raylet", "//:raylet",
"//:raylet_monitor", "//:raylet_monitor",
"//:gcs_server", "//:gcs_server",
"@plasma//:plasma_store_server", "//:plasma_store_server",
"//streaming:copy_streaming_py_proto", "//streaming:copy_streaming_py_proto",
], ],
outs = ["ray_pkg.out"], outs = ["ray_pkg.out"],
@ -1664,7 +1850,7 @@ genrule(
mkdir -p "$$WORK_DIR/python/ray/core/src/ray/gcs/redis_module/" && mkdir -p "$$WORK_DIR/python/ray/core/src/ray/gcs/redis_module/" &&
cp -f $(locations //:libray_redis_module.so) "$$WORK_DIR/python/ray/core/src/ray/gcs/redis_module/" && cp -f $(locations //:libray_redis_module.so) "$$WORK_DIR/python/ray/core/src/ray/gcs/redis_module/" &&
cp -f $(location //:raylet_monitor) "$$WORK_DIR/python/ray/core/src/ray/raylet/" && cp -f $(location //:raylet_monitor) "$$WORK_DIR/python/ray/core/src/ray/raylet/" &&
cp -f $(location @plasma//:plasma_store_server) "$$WORK_DIR/python/ray/core/src/plasma/" && cp -f $(location //:plasma_store_server) "$$WORK_DIR/python/ray/core/src/plasma/" &&
cp -f $(location //:raylet) "$$WORK_DIR/python/ray/core/src/ray/raylet/" && cp -f $(location //:raylet) "$$WORK_DIR/python/ray/core/src/ray/raylet/" &&
cp -f $(location //:gcs_server) "$$WORK_DIR/python/ray/core/src/ray/gcs/" && cp -f $(location //:gcs_server) "$$WORK_DIR/python/ray/core/src/ray/gcs/" &&
mkdir -p "$$WORK_DIR/python/ray/core/generated/ray/protocol/" && mkdir -p "$$WORK_DIR/python/ray/core/generated/ray/protocol/" &&

98
bazel/BUILD.arrow Normal file
View file

@ -0,0 +1,98 @@
load("@com_github_google_flatbuffers//:build_defs.bzl", "flatbuffer_cc_library")
# TODO(mehrdadn): (How to) support dynamic linking?
PROPAGATED_WINDOWS_DEFINES = ["ARROW_STATIC"]
COPTS = [] + select({
"@bazel_tools//src/conditions:windows": [
"-D" + "WIN32_REPLACE_FD_APIS",
"/FI" + "win32fd.h",
] + ["-D" + define for define in PROPAGATED_WINDOWS_DEFINES],
"//conditions:default": [
"-DARROW_USE_GLOG",
],
})
LINKOPTS = [] + select({
"@bazel_tools//src/conditions:windows": [
"-DefaultLib:" + "ws2_32.lib",
],
"//conditions:default": [
],
})
cc_library(
name = "arrow",
srcs = [
"cpp/src/arrow/buffer.cc",
"cpp/src/arrow/device.cc",
"cpp/src/arrow/io/interfaces.cc",
"cpp/src/arrow/io/memory.cc",
"cpp/src/arrow/memory_pool.cc",
"cpp/src/arrow/result.cc",
"cpp/src/arrow/status.cc",
"cpp/src/arrow/util/future.cc",
"cpp/src/arrow/util/io_util.cc",
"cpp/src/arrow/util/logging.cc",
"cpp/src/arrow/util/memory.cc",
"cpp/src/arrow/util/string.cc",
"cpp/src/arrow/util/string_builder.cc",
"cpp/src/arrow/util/thread_pool.cc",
"cpp/src/arrow/util/utf8.cc",
],
hdrs = [
"cpp/src/arrow/buffer.h",
"cpp/src/arrow/device.h",
"cpp/src/arrow/io/concurrency.h",
"cpp/src/arrow/io/interfaces.h",
"cpp/src/arrow/io/memory.h",
"cpp/src/arrow/io/mman.h",
"cpp/src/arrow/io/type_fwd.h",
"cpp/src/arrow/io/util_internal.h",
"cpp/src/arrow/memory_pool.h",
"cpp/src/arrow/result.h",
"cpp/src/arrow/status.h",
"cpp/src/arrow/type_fwd.h",
"cpp/src/arrow/util/bit_util.h",
"cpp/src/arrow/util/checked_cast.h",
"cpp/src/arrow/util/compare.h",
"cpp/src/arrow/util/functional.h",
"cpp/src/arrow/util/future.h",
"cpp/src/arrow/util/io_util.h",
"cpp/src/arrow/util/iterator.h",
"cpp/src/arrow/util/logging.h",
"cpp/src/arrow/util/macros.h",
"cpp/src/arrow/util/make_unique.h",
"cpp/src/arrow/util/memory.h",
"cpp/src/arrow/util/optional.h",
"cpp/src/arrow/util/string.h",
"cpp/src/arrow/util/string_builder.h",
"cpp/src/arrow/util/string_view.h",
"cpp/src/arrow/util/thread_pool.h",
"cpp/src/arrow/util/type_traits.h",
"cpp/src/arrow/util/ubsan.h",
"cpp/src/arrow/util/utf8.h",
"cpp/src/arrow/util/variant.h",
"cpp/src/arrow/util/visibility.h",
"cpp/src/arrow/util/windows_compatibility.h",
"cpp/src/arrow/util/windows_fixup.h",
"cpp/src/arrow/vendored/optional.hpp",
"cpp/src/arrow/vendored/string_view.hpp",
"cpp/src/arrow/vendored/utf8cpp/checked.h",
"cpp/src/arrow/vendored/utf8cpp/core.h",
"cpp/src/arrow/vendored/variant.hpp",
"cpp/src/arrow/vendored/xxhash.h",
"cpp/src/arrow/vendored/xxhash/xxh3.h",
"cpp/src/arrow/vendored/xxhash/xxhash.c",
"cpp/src/arrow/vendored/xxhash/xxhash.h",
],
copts = COPTS,
linkopts = LINKOPTS,
strip_include_prefix = "cpp/src",
visibility = ["//visibility:public"],
deps = [
"@//:platform_shims",
"@boost//:filesystem",
"@com_github_google_glog//:glog",
],
)

View file

@ -1,272 +0,0 @@
load("@com_github_google_flatbuffers//:build_defs.bzl", "flatbuffer_cc_library")
# TODO(mehrdadn): (How to) support dynamic linking?
PROPAGATED_WINDOWS_DEFINES = ["ARROW_STATIC"]
COPTS = [] + select({
"@bazel_tools//src/conditions:windows": [
"-D" + "WIN32_REPLACE_FD_APIS",
"/FI" + "win32fd.h",
] + ["-D" + define for define in PROPAGATED_WINDOWS_DEFINES],
"//conditions:default": [
"-DARROW_USE_GLOG",
],
})
LINKOPTS = [] + select({
"@bazel_tools//src/conditions:windows": [
"-DefaultLib:" + "ws2_32.lib",
],
"//conditions:default": [
],
})
cc_library(
name = "arrow",
srcs = [
"cpp/src/arrow/buffer.cc",
"cpp/src/arrow/device.cc",
"cpp/src/arrow/io/interfaces.cc",
"cpp/src/arrow/io/memory.cc",
"cpp/src/arrow/memory_pool.cc",
"cpp/src/arrow/result.cc",
"cpp/src/arrow/status.cc",
"cpp/src/arrow/util/future.cc",
"cpp/src/arrow/util/io_util.cc",
"cpp/src/arrow/util/logging.cc",
"cpp/src/arrow/util/memory.cc",
"cpp/src/arrow/util/string.cc",
"cpp/src/arrow/util/string_builder.cc",
"cpp/src/arrow/util/thread_pool.cc",
"cpp/src/arrow/util/utf8.cc",
],
hdrs = [
"cpp/src/arrow/buffer.h",
"cpp/src/arrow/device.h",
"cpp/src/arrow/io/concurrency.h",
"cpp/src/arrow/io/interfaces.h",
"cpp/src/arrow/io/memory.h",
"cpp/src/arrow/io/mman.h",
"cpp/src/arrow/io/type_fwd.h",
"cpp/src/arrow/io/util_internal.h",
"cpp/src/arrow/memory_pool.h",
"cpp/src/arrow/result.h",
"cpp/src/arrow/status.h",
"cpp/src/arrow/type_fwd.h",
"cpp/src/arrow/util/bit_util.h",
"cpp/src/arrow/util/checked_cast.h",
"cpp/src/arrow/util/compare.h",
"cpp/src/arrow/util/functional.h",
"cpp/src/arrow/util/future.h",
"cpp/src/arrow/util/io_util.h",
"cpp/src/arrow/util/iterator.h",
"cpp/src/arrow/util/logging.h",
"cpp/src/arrow/util/macros.h",
"cpp/src/arrow/util/make_unique.h",
"cpp/src/arrow/util/memory.h",
"cpp/src/arrow/util/optional.h",
"cpp/src/arrow/util/string.h",
"cpp/src/arrow/util/string_builder.h",
"cpp/src/arrow/util/string_view.h",
"cpp/src/arrow/util/thread_pool.h",
"cpp/src/arrow/util/type_traits.h",
"cpp/src/arrow/util/ubsan.h",
"cpp/src/arrow/util/utf8.h",
"cpp/src/arrow/util/variant.h",
"cpp/src/arrow/util/visibility.h",
"cpp/src/arrow/util/windows_compatibility.h",
"cpp/src/arrow/util/windows_fixup.h",
"cpp/src/arrow/vendored/optional.hpp",
"cpp/src/arrow/vendored/string_view.hpp",
"cpp/src/arrow/vendored/utf8cpp/checked.h",
"cpp/src/arrow/vendored/utf8cpp/core.h",
"cpp/src/arrow/vendored/variant.hpp",
"cpp/src/arrow/vendored/xxhash.h",
"cpp/src/arrow/vendored/xxhash/xxh3.h",
"cpp/src/arrow/vendored/xxhash/xxhash.c",
"cpp/src/arrow/vendored/xxhash/xxhash.h",
],
copts = COPTS,
linkopts = LINKOPTS,
strip_include_prefix = "cpp/src",
deps = [
"@//:platform_shims",
"@boost//:filesystem",
"@com_github_google_glog//:glog",
],
)
cc_library(
name = "plasma_client",
srcs = [
"cpp/src/plasma/client.cc",
"cpp/src/plasma/common.cc",
"cpp/src/plasma/fling.cc",
"cpp/src/plasma/io.cc",
"cpp/src/plasma/malloc.cc",
"cpp/src/plasma/plasma.cc",
"cpp/src/plasma/protocol.cc",
],
hdrs = [
"cpp/src/plasma/client.h",
"cpp/src/plasma/common.h",
"cpp/src/plasma/common_generated.h",
"cpp/src/plasma/compat.h",
"cpp/src/plasma/external_store.h",
"cpp/src/plasma/fling.h",
"cpp/src/plasma/io.h",
"cpp/src/plasma/malloc.h",
"cpp/src/plasma/plasma.h",
"cpp/src/plasma/plasma_generated.h",
"cpp/src/plasma/protocol.h",
],
copts = COPTS,
defines = select({
"@bazel_tools//src/conditions:windows": PROPAGATED_WINDOWS_DEFINES,
"//conditions:default": [],
}),
linkopts = LINKOPTS,
strip_include_prefix = "cpp/src",
visibility = ["//visibility:public"],
deps = [
":arrow",
":common_fbs",
":plasma_fbs",
"@//:platform_shims",
"@com_github_google_glog//:glog",
],
)
cc_binary(
name = "libplasma_java.so",
srcs = [
"cpp/src/plasma/lib/java/org_apache_arrow_plasma_PlasmaClientJNI.cc",
"cpp/src/plasma/lib/java/org_apache_arrow_plasma_PlasmaClientJNI.h",
":jni.h",
":jni_md.h",
],
includes = [
".",
"cpp/src",
],
linkshared = 1,
linkstatic = 1,
deps = [":plasma_client"],
)
genrule(
name = "copy_jni_h",
srcs = ["@bazel_tools//tools/jdk:jni_header"],
outs = ["jni.h"],
cmd = "cp -f $< $@",
)
genrule(
name = "copy_jni_md_h",
srcs = select({
"@bazel_tools//src/conditions:windows": ["@bazel_tools//tools/jdk:jni_md_header-windows"],
"@bazel_tools//src/conditions:darwin": ["@bazel_tools//tools/jdk:jni_md_header-darwin"],
"//conditions:default": ["@bazel_tools//tools/jdk:jni_md_header-linux"],
}),
outs = ["jni_md.h"],
cmd = "cp -f $< $@",
)
genrule(
name = "plasma-jni-darwin-compat",
srcs = [":libplasma_java.so"],
outs = ["libplasma_java.dylib"],
cmd = "cp $< $@",
output_to_bindir = 1,
)
cc_library(
name = "ae",
srcs = [
"cpp/src/plasma/thirdparty/ae/ae.c",
],
hdrs = [
"cpp/src/plasma/thirdparty/ae/ae.h",
"cpp/src/plasma/thirdparty/ae/ae_epoll.c",
"cpp/src/plasma/thirdparty/ae/ae_evport.c",
"cpp/src/plasma/thirdparty/ae/ae_kqueue.c",
"cpp/src/plasma/thirdparty/ae/ae_select.c",
"cpp/src/plasma/thirdparty/ae/config.h",
"cpp/src/plasma/thirdparty/ae/zmalloc.h",
],
copts = COPTS,
includes = [
"cpp/src/plasma/thirdparty/ae",
],
strip_include_prefix = "cpp/src",
visibility = ["//visibility:public"],
deps = [
"@//:platform_shims",
],
)
cc_library(
name = "plasma_lib",
srcs = [
"cpp/src/plasma/dlmalloc.cc",
"cpp/src/plasma/events.cc",
"cpp/src/plasma/eviction_policy.cc",
"cpp/src/plasma/external_store.cc",
"cpp/src/plasma/plasma_allocator.cc",
"cpp/src/plasma/quota_aware_policy.cc",
],
hdrs = [
"cpp/src/plasma/events.h",
"cpp/src/plasma/eviction_policy.h",
"cpp/src/plasma/external_store.h",
"cpp/src/plasma/plasma_allocator.h",
"cpp/src/plasma/quota_aware_policy.h",
"cpp/src/plasma/store.h",
"cpp/src/plasma/thirdparty/dlmalloc.c",
],
copts = COPTS,
linkopts = LINKOPTS,
strip_include_prefix = "cpp/src",
deps = [
":ae",
":plasma_client",
"@//:platform_shims",
"@com_github_google_glog//:glog",
],
)
cc_binary(
name = "plasma_store_server",
srcs = [
"cpp/src/plasma/store.cc",
],
copts = COPTS,
visibility = ["//visibility:public"],
deps = [
":plasma_lib",
"@//:platform_shims",
],
)
FLATC_ARGS = [
"--gen-object-api",
"--gen-mutable",
"--scoped-enums",
]
flatbuffer_cc_library(
name = "common_fbs",
srcs = ["cpp/src/plasma/common.fbs"],
flatc_args = FLATC_ARGS,
out_prefix = "cpp/src/plasma/",
)
flatbuffer_cc_library(
name = "plasma_fbs",
srcs = ["cpp/src/plasma/plasma.fbs"],
flatc_args = FLATC_ARGS,
includes = ["cpp/src/plasma/common.fbs"],
out_prefix = "cpp/src/plasma/",
)
exports_files(["cpp/src/plasma/common.fbs"])

View file

@ -162,18 +162,13 @@ def ray_deps_setup():
) )
auto_http_archive( auto_http_archive(
name = "plasma", name = "arrow",
build_file = True, build_file = True,
url = "https://github.com/apache/arrow/archive/af45b9212156980f55c399e2e88b4e19b4bb8ec1.tar.gz", url = "https://github.com/apache/arrow/archive/af45b9212156980f55c399e2e88b4e19b4bb8ec1.tar.gz",
sha256 = "2f0aaa50053792aa274b402f2530e63c1542085021cfef83beee9281412c12f6", sha256 = "2f0aaa50053792aa274b402f2530e63c1542085021cfef83beee9281412c12f6",
patches = [ patches = [
"//thirdparty/patches:arrow-headers-unused.patch",
"//thirdparty/patches:arrow-windows-export.patch", "//thirdparty/patches:arrow-windows-export.patch",
"//thirdparty/patches:arrow-windows-nonstdc.patch", "//thirdparty/patches:arrow-windows-nonstdc.patch",
"//thirdparty/patches:arrow-windows-sigpipe.patch",
"//thirdparty/patches:arrow-windows-socket.patch",
"//thirdparty/patches:arrow-windows-dlmalloc.patch",
"//thirdparty/patches:arrow-windows-tcp.patch",
], ],
) )

View file

@ -44,7 +44,7 @@ while [[ $# > 0 ]]; do
done done
pushd $ROOT_DIR/../.. pushd $ROOT_DIR/../..
BAZEL_FILES="bazel/BUILD bazel/BUILD.plasma bazel/ray.bzl BUILD.bazel java/BUILD.bazel BAZEL_FILES="bazel/BUILD bazel/BUILD.arrow bazel/ray.bzl BUILD.bazel java/BUILD.bazel
cpp/BUILD.bazel streaming/BUILD.bazel streaming/java/BUILD.bazel WORKSPACE" cpp/BUILD.bazel streaming/BUILD.bazel streaming/java/BUILD.bazel WORKSPACE"
buildifier -mode=$RUN_TYPE -diff_command="diff -u" $BAZEL_FILES buildifier -mode=$RUN_TYPE -diff_command="diff -u" $BAZEL_FILES
popd popd

View file

@ -146,11 +146,11 @@ filegroup(
genrule( genrule(
name = "cp_plasma_store_server", name = "cp_plasma_store_server",
srcs = [ srcs = [
"@plasma//:plasma_store_server", "//:plasma_store_server",
], ],
outs = ["plasma_store_server"], outs = ["plasma_store_server"],
cmd = """ cmd = """
cp -f $(location @plasma//:plasma_store_server) $@ cp -f $(location //:plasma_store_server) $@
""", """,
) )

View file

@ -0,0 +1 @@
DisableFormat: true

18
src/ray/plasma/.gitignore vendored Normal file
View file

@ -0,0 +1,18 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
*_generated.h

1253
src/ray/plasma/client.cc Normal file

File diff suppressed because it is too large Load diff

312
src/ray/plasma/client.h Normal file
View file

@ -0,0 +1,312 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_CLIENT_H
#define PLASMA_CLIENT_H
#include <functional>
#include <memory>
#include <string>
#include <vector>
#include "arrow/buffer.h"
#include "arrow/status.h"
#include "arrow/util/macros.h"
#include "arrow/util/visibility.h"
#include "plasma/common.h"
using arrow::Buffer;
using arrow::Status;
namespace plasma {
/// Object buffer data structure.
struct ObjectBuffer {
/// The data buffer.
std::shared_ptr<Buffer> data;
/// The metadata buffer.
std::shared_ptr<Buffer> metadata;
/// The device number.
int device_num;
};
class ARROW_EXPORT PlasmaClient {
public:
PlasmaClient();
~PlasmaClient();
/// Connect to the local plasma store. Return the resulting connection.
///
/// \param store_socket_name The name of the UNIX domain socket to use to
/// connect to the Plasma store.
/// \param manager_socket_name The name of the UNIX domain socket to use to
/// connect to the local Plasma manager. If this is "", then this
/// function will not connect to a manager.
/// Note that plasma manager is no longer supported, this function
/// will return failure if this is not "".
/// \param release_delay Deprecated (not used).
/// \param num_retries number of attempts to connect to IPC socket, default 50
/// \return The return status.
Status Connect(const std::string& store_socket_name,
const std::string& manager_socket_name = "", int release_delay = 0,
int num_retries = -1);
/// Set runtime options for this client.
///
/// \param client_name The name of the client, used in debug messages.
/// \param output_memory_quota The memory quota in bytes for objects created by
/// this client.
Status SetClientOptions(const std::string& client_name, int64_t output_memory_quota);
/// Create an object in the Plasma Store. Any metadata for this object must be
/// be passed in when the object is created.
///
/// \param object_id The ID to use for the newly created object.
/// \param data_size The size in bytes of the space to be allocated for this
/// object's
/// data (this does not include space used for metadata).
/// \param metadata The object's metadata. If there is no metadata, this
/// pointer
/// should be NULL.
/// \param metadata_size The size in bytes of the metadata. If there is no
/// metadata, this should be 0.
/// \param data The address of the newly created object will be written here.
/// \param device_num The number of the device where the object is being
/// created.
/// device_num = 0 corresponds to the host,
/// device_num = 1 corresponds to GPU0,
/// device_num = 2 corresponds to GPU1, etc.
/// \param evict_if_full Whether to evict other objects to make space for
/// this object.
/// \return The return status.
///
/// The returned object must be released once it is done with. It must also
/// be either sealed or aborted.
Status Create(const ObjectID& object_id, int64_t data_size, const uint8_t* metadata,
int64_t metadata_size, std::shared_ptr<Buffer>* data, int device_num = 0,
bool evict_if_full = true);
/// Create and seal an object in the object store. This is an optimization
/// which allows small objects to be created quickly with fewer messages to
/// the store.
///
/// \param object_id The ID of the object to create.
/// \param data The data for the object to create.
/// \param metadata The metadata for the object to create.
/// \param evict_if_full Whether to evict other objects to make space for
/// this object.
/// \return The return status.
Status CreateAndSeal(const ObjectID& object_id, const std::string& data,
const std::string& metadata, bool evict_if_full = true);
/// Create and seal multiple objects in the object store. This is an optimization
/// of CreateAndSeal to eliminate the cost of IPC per object.
///
/// \param object_ids The vector of IDs of the objects to create.
/// \param data The vector of data for the objects to create.
/// \param metadata The vector of metadata for the objects to create.
/// \param evict_if_full Whether to evict other objects to make space for
/// these objects.
/// \return The return status.
Status CreateAndSealBatch(const std::vector<ObjectID>& object_ids,
const std::vector<std::string>& data,
const std::vector<std::string>& metadata,
bool evict_if_full = true);
/// Get some objects from the Plasma Store. This function will block until the
/// objects have all been created and sealed in the Plasma Store or the
/// timeout expires.
///
/// If an object was not retrieved, the corresponding metadata and data
/// fields in the ObjectBuffer structure will evaluate to false.
/// Objects are automatically released by the client when their buffers
/// get out of scope.
///
/// \param object_ids The IDs of the objects to get.
/// \param timeout_ms The amount of time in milliseconds to wait before this
/// request times out. If this value is -1, then no timeout is set.
/// \param[out] object_buffers The object results.
/// \return The return status.
Status Get(const std::vector<ObjectID>& object_ids, int64_t timeout_ms,
std::vector<ObjectBuffer>* object_buffers);
/// Deprecated variant of Get() that doesn't automatically release buffers
/// when they get out of scope.
///
/// \param object_ids The IDs of the objects to get.
/// \param num_objects The number of object IDs to get.
/// \param timeout_ms The amount of time in milliseconds to wait before this
/// request times out. If this value is -1, then no timeout is set.
/// \param object_buffers An array where the results will be stored.
/// \return The return status.
///
/// The caller is responsible for releasing any retrieved objects, but it
/// should not release objects that were not retrieved.
Status Get(const ObjectID* object_ids, int64_t num_objects, int64_t timeout_ms,
ObjectBuffer* object_buffers);
/// Tell Plasma that the client no longer needs the object. This should be
/// called after Get() or Create() when the client is done with the object.
/// After this call, the buffer returned by Get() is no longer valid.
///
/// \param object_id The ID of the object that is no longer needed.
/// \return The return status.
Status Release(const ObjectID& object_id);
/// Check if the object store contains a particular object and the object has
/// been sealed. The result will be stored in has_object.
///
/// @todo: We may want to indicate if the object has been created but not
/// sealed.
///
/// \param object_id The ID of the object whose presence we are checking.
/// \param has_object The function will write true at this address if
/// the object is present and false if it is not present.
/// \return The return status.
Status Contains(const ObjectID& object_id, bool* has_object);
/// List all the objects in the object store.
///
/// This API is experimental and might change in the future.
///
/// \param[out] objects ObjectTable of objects in the store. For each entry
/// in the map, the following fields are available:
/// - metadata_size: Size of the object metadata in bytes
/// - data_size: Size of the object data in bytes
/// - ref_count: Number of clients referencing the object buffer
/// - create_time: Unix timestamp of the object creation
/// - construct_duration: Object creation time in seconds
/// - state: Is the object still being created or already sealed?
/// \return The return status.
Status List(ObjectTable* objects);
/// Abort an unsealed object in the object store. If the abort succeeds, then
/// it will be as if the object was never created at all. The unsealed object
/// must have only a single reference (the one that would have been removed by
/// calling Seal).
///
/// \param object_id The ID of the object to abort.
/// \return The return status.
Status Abort(const ObjectID& object_id);
/// Seal an object in the object store. The object will be immutable after
/// this
/// call.
///
/// \param object_id The ID of the object to seal.
/// \return The return status.
Status Seal(const ObjectID& object_id);
/// Delete an object from the object store. This currently assumes that the
/// object is present, has been sealed and not used by another client. Otherwise,
/// it is a no operation.
///
/// \todo We may want to allow the deletion of objects that are not present or
/// haven't been sealed.
///
/// \param object_id The ID of the object to delete.
/// \return The return status.
Status Delete(const ObjectID& object_id);
/// Delete a list of objects from the object store. This currently assumes that the
/// object is present, has been sealed and not used by another client. Otherwise,
/// it is a no operation.
///
/// \param object_ids The list of IDs of the objects to delete.
/// \return The return status. If all the objects are non-existent, return OK.
Status Delete(const std::vector<ObjectID>& object_ids);
/// Delete objects until we have freed up num_bytes bytes or there are no more
/// released objects that can be deleted.
///
/// \param num_bytes The number of bytes to try to free up.
/// \param num_bytes_evicted Out parameter for total number of bytes of space
/// retrieved.
/// \return The return status.
Status Evict(int64_t num_bytes, int64_t& num_bytes_evicted);
/// Bump objects up in the LRU cache, i.e. treat them as recently accessed.
/// Objects that do not exist in the store will be ignored.
///
/// \param object_ids The IDs of the objects to bump.
/// \return The return status.
Status Refresh(const std::vector<ObjectID>& object_ids);
/// Compute the hash of an object in the object store.
///
/// \param object_id The ID of the object we want to hash.
/// \param digest A pointer at which to return the hash digest of the object.
/// The pointer must have at least kDigestSize bytes allocated.
/// \return The return status.
Status Hash(const ObjectID& object_id, uint8_t* digest);
/// Subscribe to notifications when objects are sealed in the object store.
/// Whenever an object is sealed, a message will be written to the client
/// socket that is returned by this method.
///
/// \param fd Out parameter for the file descriptor the client should use to
/// read notifications
/// from the object store about sealed objects.
/// \return The return status.
Status Subscribe(int* fd);
/// Receive next object notification for this client if Subscribe has been called.
///
/// \param fd The file descriptor we are reading the notification from.
/// \param object_id Out parameter, the object_id of the object that was sealed.
/// \param data_size Out parameter, the data size of the object that was sealed.
/// \param metadata_size Out parameter, the metadata size of the object that was sealed.
/// \return The return status.
Status GetNotification(int fd, ObjectID* object_id, int64_t* data_size,
int64_t* metadata_size);
Status DecodeNotifications(const uint8_t* buffer, std::vector<ObjectID>* object_ids,
std::vector<int64_t>* data_sizes,
std::vector<int64_t>* metadata_sizes);
/// Disconnect from the local plasma instance, including the local store and
/// manager.
///
/// \return The return status.
Status Disconnect();
/// Get the current debug string from the plasma store server.
///
/// \return The debug string.
std::string DebugString();
/// Get the memory capacity of the store.
///
/// \return Memory capacity of the store in bytes.
int64_t store_capacity();
private:
friend class PlasmaBuffer;
friend class PlasmaMutableBuffer;
FRIEND_TEST(TestPlasmaStore, GetTest);
FRIEND_TEST(TestPlasmaStore, LegacyGetTest);
FRIEND_TEST(TestPlasmaStore, AbortTest);
bool IsInUse(const ObjectID& object_id);
class ARROW_NO_EXPORT Impl;
std::shared_ptr<Impl> impl_;
};
} // namespace plasma
#endif // PLASMA_CLIENT_H

195
src/ray/plasma/common.cc Normal file
View file

@ -0,0 +1,195 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/common.h"
#include <limits>
#include <utility>
#include "arrow/util/ubsan.h"
#include "plasma/plasma_generated.h"
namespace fb = plasma::flatbuf;
namespace plasma {
namespace {
const char kErrorDetailTypeId[] = "plasma::PlasmaStatusDetail";
class PlasmaStatusDetail : public arrow::StatusDetail {
public:
explicit PlasmaStatusDetail(PlasmaErrorCode code) : code_(code) {}
const char* type_id() const override { return kErrorDetailTypeId; }
std::string ToString() const override {
const char* type;
switch (code()) {
case PlasmaErrorCode::PlasmaObjectExists:
type = "Plasma object exists";
break;
case PlasmaErrorCode::PlasmaObjectNonexistent:
type = "Plasma object is nonexistent";
break;
case PlasmaErrorCode::PlasmaStoreFull:
type = "Plasma store is full";
break;
case PlasmaErrorCode::PlasmaObjectAlreadySealed:
type = "Plasma object is already sealed";
break;
default:
type = "Unknown plasma error";
break;
}
return std::string(type);
}
PlasmaErrorCode code() const { return code_; }
private:
PlasmaErrorCode code_;
};
bool IsPlasmaStatus(const arrow::Status& status, PlasmaErrorCode code) {
if (status.ok()) {
return false;
}
auto* detail = status.detail().get();
return detail != nullptr && detail->type_id() == kErrorDetailTypeId &&
static_cast<PlasmaStatusDetail*>(detail)->code() == code;
}
} // namespace
using arrow::Status;
arrow::Status MakePlasmaError(PlasmaErrorCode code, std::string message) {
arrow::StatusCode arrow_code = arrow::StatusCode::UnknownError;
switch (code) {
case PlasmaErrorCode::PlasmaObjectExists:
arrow_code = arrow::StatusCode::AlreadyExists;
break;
case PlasmaErrorCode::PlasmaObjectNonexistent:
arrow_code = arrow::StatusCode::KeyError;
break;
case PlasmaErrorCode::PlasmaStoreFull:
arrow_code = arrow::StatusCode::CapacityError;
break;
case PlasmaErrorCode::PlasmaObjectAlreadySealed:
// Maybe a stretch?
arrow_code = arrow::StatusCode::TypeError;
break;
}
return arrow::Status(arrow_code, std::move(message),
std::make_shared<PlasmaStatusDetail>(code));
}
bool IsPlasmaObjectExists(const arrow::Status& status) {
return IsPlasmaStatus(status, PlasmaErrorCode::PlasmaObjectExists);
}
bool IsPlasmaObjectNonexistent(const arrow::Status& status) {
return IsPlasmaStatus(status, PlasmaErrorCode::PlasmaObjectNonexistent);
}
bool IsPlasmaObjectAlreadySealed(const arrow::Status& status) {
return IsPlasmaStatus(status, PlasmaErrorCode::PlasmaObjectAlreadySealed);
}
bool IsPlasmaStoreFull(const arrow::Status& status) {
return IsPlasmaStatus(status, PlasmaErrorCode::PlasmaStoreFull);
}
UniqueID UniqueID::from_binary(const std::string& binary) {
UniqueID id;
std::memcpy(&id, binary.data(), sizeof(id));
return id;
}
const uint8_t* UniqueID::data() const { return id_; }
uint8_t* UniqueID::mutable_data() { return id_; }
std::string UniqueID::binary() const {
return std::string(reinterpret_cast<const char*>(id_), kUniqueIDSize);
}
std::string UniqueID::hex() const {
constexpr char hex[] = "0123456789abcdef";
std::string result;
for (int i = 0; i < kUniqueIDSize; i++) {
unsigned int val = id_[i];
result.push_back(hex[val >> 4]);
result.push_back(hex[val & 0xf]);
}
return result;
}
// This code is from https://sites.google.com/site/murmurhash/
// and is public domain.
uint64_t MurmurHash64A(const void* key, int len, unsigned int seed) {
const uint64_t m = 0xc6a4a7935bd1e995;
const int r = 47;
uint64_t h = seed ^ (len * m);
const uint64_t* data = reinterpret_cast<const uint64_t*>(key);
const uint64_t* end = data + (len / 8);
while (data != end) {
uint64_t k = arrow::util::SafeLoad(data++);
k *= m;
k ^= k >> r;
k *= m;
h ^= k;
h *= m;
}
const unsigned char* data2 = reinterpret_cast<const unsigned char*>(data);
switch (len & 7) {
case 7:
h ^= uint64_t(data2[6]) << 48; // fall through
case 6:
h ^= uint64_t(data2[5]) << 40; // fall through
case 5:
h ^= uint64_t(data2[4]) << 32; // fall through
case 4:
h ^= uint64_t(data2[3]) << 24; // fall through
case 3:
h ^= uint64_t(data2[2]) << 16; // fall through
case 2:
h ^= uint64_t(data2[1]) << 8; // fall through
case 1:
h ^= uint64_t(data2[0]);
h *= m;
}
h ^= h >> r;
h *= m;
h ^= h >> r;
return h;
}
size_t UniqueID::hash() const { return MurmurHash64A(&id_[0], kUniqueIDSize, 0); }
bool UniqueID::operator==(const UniqueID& rhs) const {
return std::memcmp(data(), rhs.data(), kUniqueIDSize) == 0;
}
const PlasmaStoreInfo* plasma_config;
} // namespace plasma

39
src/ray/plasma/common.fbs Normal file
View file

@ -0,0 +1,39 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
namespace plasma.flatbuf;
// Object information data structure.
table ObjectInfo {
// Object ID of this object.
object_id: string;
// Number of bytes the content of this object occupies in memory.
data_size: long;
// Number of bytes the metadata of this object occupies in memory.
metadata_size: long;
// Number of clients using the objects.
ref_count: int;
// Unix epoch of when this object was created.
create_time: long;
// How long creation of this object took.
construct_duration: long;
// Hash of the object content. If the object is not sealed yet this is
// an empty string.
digest: string;
// Specifies if this object was deleted or added.
is_deletion: bool;
}

158
src/ray/plasma/common.h Normal file
View file

@ -0,0 +1,158 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_COMMON_H
#define PLASMA_COMMON_H
#include <stddef.h>
#include <cstring>
#include <memory>
#include <string>
// TODO(pcm): Convert getopt and sscanf in the store to use more idiomatic C++
// and get rid of the next three lines:
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <unordered_map>
#include "plasma/compat.h"
#include "arrow/status.h"
#ifdef PLASMA_CUDA
#include "arrow/gpu/cuda_api.h"
#endif
namespace plasma {
enum class ObjectLocation : int32_t { Local, Remote, Nonexistent };
enum class PlasmaErrorCode : int8_t {
PlasmaObjectExists = 1,
PlasmaObjectNonexistent = 2,
PlasmaStoreFull = 3,
PlasmaObjectAlreadySealed = 4,
};
ARROW_EXPORT arrow::Status MakePlasmaError(PlasmaErrorCode code, std::string message);
/// Return true iff the status indicates an already existing Plasma object.
ARROW_EXPORT bool IsPlasmaObjectExists(const arrow::Status& status);
/// Return true iff the status indicates a non-existent Plasma object.
ARROW_EXPORT bool IsPlasmaObjectNonexistent(const arrow::Status& status);
/// Return true iff the status indicates an already sealed Plasma object.
ARROW_EXPORT bool IsPlasmaObjectAlreadySealed(const arrow::Status& status);
/// Return true iff the status indicates the Plasma store reached its capacity limit.
ARROW_EXPORT bool IsPlasmaStoreFull(const arrow::Status& status);
constexpr int64_t kUniqueIDSize = 20;
class ARROW_EXPORT UniqueID {
public:
static UniqueID from_binary(const std::string& binary);
bool operator==(const UniqueID& rhs) const;
const uint8_t* data() const;
uint8_t* mutable_data();
std::string binary() const;
std::string hex() const;
size_t hash() const;
static int64_t size() { return kUniqueIDSize; }
private:
uint8_t id_[kUniqueIDSize];
};
static_assert(std::is_pod<UniqueID>::value, "UniqueID must be plain old data");
typedef UniqueID ObjectID;
/// Size of object hash digests.
constexpr int64_t kDigestSize = sizeof(uint64_t);
enum class ObjectState : int {
/// Object was created but not sealed in the local Plasma Store.
PLASMA_CREATED = 1,
/// Object is sealed and stored in the local Plasma Store.
PLASMA_SEALED = 2,
/// Object is evicted to external store.
PLASMA_EVICTED = 3,
};
namespace internal {
struct CudaIpcPlaceholder {};
} // namespace internal
/// This type is used by the Plasma store. It is here because it is exposed to
/// the eviction policy.
struct ObjectTableEntry {
ObjectTableEntry();
~ObjectTableEntry();
/// Memory mapped file containing the object.
int fd;
/// Device number.
int device_num;
/// Size of the underlying map.
int64_t map_size;
/// Offset from the base of the mmap.
ptrdiff_t offset;
/// Pointer to the object data. Needed to free the object.
uint8_t* pointer;
/// Size of the object in bytes.
int64_t data_size;
/// Size of the object metadata in bytes.
int64_t metadata_size;
/// Number of clients currently using this object.
int ref_count;
/// Unix epoch of when this object was created.
int64_t create_time;
/// How long creation of this object took.
int64_t construct_duration;
/// The state of the object, e.g., whether it is open or sealed.
ObjectState state;
/// The digest of the object. Used to see if two objects are the same.
unsigned char digest[kDigestSize];
#ifdef PLASMA_CUDA
/// IPC GPU handle to share with clients.
std::shared_ptr<::arrow::cuda::CudaIpcMemHandle> ipc_handle;
#else
std::shared_ptr<internal::CudaIpcPlaceholder> ipc_handle;
#endif
};
/// Mapping from ObjectIDs to information about the object.
typedef std::unordered_map<ObjectID, std::unique_ptr<ObjectTableEntry>> ObjectTable;
/// Globally accessible reference to plasma store configuration.
/// TODO(pcm): This can be avoided with some refactoring of existing code
/// by making it possible to pass a context object through dlmalloc.
struct PlasmaStoreInfo;
extern const PlasmaStoreInfo* plasma_config;
} // namespace plasma
namespace std {
template <>
struct hash<::plasma::UniqueID> {
size_t operator()(const ::plasma::UniqueID& id) const { return id.hash(); }
};
} // namespace std
#endif // PLASMA_COMMON_H

35
src/ray/plasma/compat.h Normal file
View file

@ -0,0 +1,35 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_COMPAT_H
#define PLASMA_COMPAT_H
// Workaround for multithreading on XCode 9, see
// https://issues.apache.org/jira/browse/ARROW-1622 and
// https://github.com/tensorflow/tensorflow/issues/13220#issuecomment-331579775
// This should be a short-term fix until the problem is fixed upstream.
#ifdef __APPLE__
#ifndef _MACH_PORT_T
#define _MACH_PORT_T
#include <sys/_types.h> /* __darwin_mach_port_t */
typedef __darwin_mach_port_t mach_port_t;
#include <pthread.h>
mach_port_t pthread_mach_thread_np(pthread_t);
#endif /* _MACH_PORT_T */
#endif /* __APPLE__ */
#endif // PLASMA_COMPAT_H

186
src/ray/plasma/dlmalloc.cc Normal file
View file

@ -0,0 +1,186 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/malloc.h"
#include <assert.h>
#include <stddef.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#ifdef _WIN32
#include <Windows.h>
#else
#include <sys/mman.h>
#include <unistd.h>
#endif
#include <cerrno>
#include <string>
#include <vector>
#include "plasma/common.h"
#include "plasma/plasma.h"
namespace plasma {
void* fake_mmap(size_t);
int fake_munmap(void*, int64_t);
#define MMAP(s) fake_mmap(s)
#define MUNMAP(a, s) fake_munmap(a, s)
#define DIRECT_MMAP(s) fake_mmap(s)
#define DIRECT_MUNMAP(a, s) fake_munmap(a, s)
#define USE_DL_PREFIX
#define HAVE_MORECORE 0
#define DEFAULT_MMAP_THRESHOLD MAX_SIZE_T
#define DEFAULT_GRANULARITY ((size_t)128U * 1024U)
#include "plasma/thirdparty/dlmalloc.c" // NOLINT
#undef MMAP
#undef MUNMAP
#undef DIRECT_MMAP
#undef DIRECT_MUNMAP
#undef USE_DL_PREFIX
#undef HAVE_MORECORE
#undef DEFAULT_GRANULARITY
// dlmalloc.c defined DEBUG which will conflict with ARROW_LOG(DEBUG).
#ifdef DEBUG
#undef DEBUG
#endif
constexpr int GRANULARITY_MULTIPLIER = 2;
static void* pointer_advance(void* p, ptrdiff_t n) { return (unsigned char*)p + n; }
static void* pointer_retreat(void* p, ptrdiff_t n) { return (unsigned char*)p - n; }
// Create a buffer. This is creating a temporary file and then
// immediately unlinking it so we do not leave traces in the system.
int create_buffer(int64_t size) {
int fd;
std::string file_template = plasma_config->directory;
#ifdef _WIN32
HANDLE h = CreateFileMapping(INVALID_HANDLE_VALUE, NULL, PAGE_READWRITE,
(DWORD)((uint64_t)size >> (CHAR_BIT * sizeof(DWORD))),
(DWORD)(uint64_t)size, NULL);
if (h) {
fd = fh_open(reinterpret_cast<intptr_t>(h), -1);
} else {
fd = -1;
}
#else
file_template += "/plasmaXXXXXX";
std::vector<char> file_name(file_template.begin(), file_template.end());
file_name.push_back('\0');
fd = mkstemp(&file_name[0]);
if (fd < 0) {
ARROW_LOG(FATAL) << "create_buffer failed to open file " << &file_name[0];
return -1;
}
// Immediately unlink the file so we do not leave traces in the system.
if (unlink(&file_name[0]) != 0) {
ARROW_LOG(FATAL) << "failed to unlink file " << &file_name[0];
return -1;
}
if (!plasma_config->hugepages_enabled) {
// Increase the size of the file to the desired size. This seems not to be
// needed for files that are backed by the huge page fs, see also
// http://www.mail-archive.com/kvm-devel@lists.sourceforge.net/msg14737.html
if (ftruncate(fd, (off_t)size) != 0) {
ARROW_LOG(FATAL) << "failed to ftruncate file " << &file_name[0];
return -1;
}
}
#endif
return fd;
}
void* fake_mmap(size_t size) {
// Add kMmapRegionsGap so that the returned pointer is deliberately not
// page-aligned. This ensures that the segments of memory returned by
// fake_mmap are never contiguous.
size += kMmapRegionsGap;
int fd = create_buffer(size);
ARROW_CHECK(fd >= 0) << "Failed to create buffer during mmap";
// MAP_POPULATE can be used to pre-populate the page tables for this memory region
// which avoids work when accessing the pages later. However it causes long pauses
// when mmapping the files. Only supported on Linux.
void* pointer;
#ifdef _WIN32
pointer = MapViewOfFile(reinterpret_cast<HANDLE>(fh_get(fd)), FILE_MAP_ALL_ACCESS, 0, 0, size);
if (pointer == NULL) {
ARROW_LOG(ERROR) << "MapViewOfFile failed with error: " << GetLastError();
return reinterpret_cast<void*>(-1);
}
#else
pointer = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
if (pointer == MAP_FAILED) {
ARROW_LOG(ERROR) << "mmap failed with error: " << std::strerror(errno);
if (errno == ENOMEM && plasma_config->hugepages_enabled) {
ARROW_LOG(ERROR)
<< " (this probably means you have to increase /proc/sys/vm/nr_hugepages)";
}
return pointer;
}
#endif
// Increase dlmalloc's allocation granularity directly.
mparams.granularity *= GRANULARITY_MULTIPLIER;
MmapRecord& record = mmap_records[pointer];
record.fd = fd;
record.size = size;
// We lie to dlmalloc about where mapped memory actually lives.
pointer = pointer_advance(pointer, kMmapRegionsGap);
ARROW_LOG(DEBUG) << pointer << " = fake_mmap(" << size << ")";
return pointer;
}
int fake_munmap(void* addr, int64_t size) {
ARROW_LOG(DEBUG) << "fake_munmap(" << addr << ", " << size << ")";
addr = pointer_retreat(addr, kMmapRegionsGap);
size += kMmapRegionsGap;
auto entry = mmap_records.find(addr);
if (entry == mmap_records.end() || entry->second.size != size) {
// Reject requests to munmap that don't directly match previous
// calls to mmap, to prevent dlmalloc from trimming.
return -1;
}
int r;
#ifdef _WIN32
r = UnmapViewOfFile(addr) ? 0 : -1;
#else
r = munmap(addr, size);
#endif
if (r == 0) {
close(entry->second.fd);
}
mmap_records.erase(entry);
return r;
}
void SetMallocGranularity(int value) { change_mparam(M_GRANULARITY, value); }
} // namespace plasma

107
src/ray/plasma/events.cc Normal file
View file

@ -0,0 +1,107 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/events.h"
#include <utility>
#include <errno.h>
extern "C" {
#include "plasma/thirdparty/ae/ae.h"
}
namespace plasma {
// Verify that the constants defined in events.h are defined correctly.
static_assert(kEventLoopTimerDone == AE_NOMORE, "constant defined incorrectly");
static_assert(kEventLoopOk == AE_OK, "constant defined incorrectly");
static_assert(kEventLoopRead == AE_READABLE, "constant defined incorrectly");
static_assert(kEventLoopWrite == AE_WRITABLE, "constant defined incorrectly");
void EventLoop::FileEventCallback(aeEventLoop* loop, int fd, void* context, int events) {
FileCallback* callback = reinterpret_cast<FileCallback*>(context);
(*callback)(events);
}
int EventLoop::TimerEventCallback(aeEventLoop* loop, TimerID timer_id, void* context) {
TimerCallback* callback = reinterpret_cast<TimerCallback*>(context);
return (*callback)(timer_id);
}
constexpr int kInitialEventLoopSize = 1024;
EventLoop::EventLoop() { loop_ = aeCreateEventLoop(kInitialEventLoopSize); }
bool EventLoop::AddFileEvent(int fd, int events, const FileCallback& callback) {
if (file_callbacks_.find(fd) != file_callbacks_.end()) {
return false;
}
auto data = std::unique_ptr<FileCallback>(new FileCallback(callback));
void* context = reinterpret_cast<void*>(data.get());
// Try to add the file descriptor.
int err = aeCreateFileEvent(loop_, fd, events, EventLoop::FileEventCallback, context);
// If it cannot be added, increase the size of the event loop.
if (err == AE_ERR && errno == ERANGE) {
err = aeResizeSetSize(loop_, 3 * aeGetSetSize(loop_) / 2);
if (err != AE_OK) {
return false;
}
err = aeCreateFileEvent(loop_, fd, events, EventLoop::FileEventCallback, context);
}
// In any case, test if there were errors.
if (err == AE_OK) {
file_callbacks_.emplace(fd, std::move(data));
return true;
}
return false;
}
void EventLoop::RemoveFileEvent(int fd) {
aeDeleteFileEvent(loop_, fd, AE_READABLE | AE_WRITABLE);
file_callbacks_.erase(fd);
}
void EventLoop::Start() { aeMain(loop_); }
void EventLoop::Stop() { aeStop(loop_); }
void EventLoop::Shutdown() {
if (loop_ != nullptr) {
aeDeleteEventLoop(loop_);
loop_ = nullptr;
}
}
EventLoop::~EventLoop() { Shutdown(); }
int64_t EventLoop::AddTimer(int64_t timeout, const TimerCallback& callback) {
auto data = std::unique_ptr<TimerCallback>(new TimerCallback(callback));
void* context = reinterpret_cast<void*>(data.get());
int64_t timer_id =
aeCreateTimeEvent(loop_, timeout, EventLoop::TimerEventCallback, context, NULL);
timer_callbacks_.emplace(timer_id, std::move(data));
return timer_id;
}
int EventLoop::RemoveTimer(int64_t timer_id) {
int err = aeDeleteTimeEvent(loop_, timer_id);
timer_callbacks_.erase(timer_id);
return err;
}
} // namespace plasma

111
src/ray/plasma/events.h Normal file
View file

@ -0,0 +1,111 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_EVENTS
#define PLASMA_EVENTS
#include <functional>
#include <memory>
#include <unordered_map>
struct aeEventLoop;
namespace plasma {
// The constants below are defined using hardcoded values taken from ae.h so
// that ae.h does not need to be included in this file.
/// Constant specifying that the timer is done and it will be removed.
constexpr int kEventLoopTimerDone = -1; // AE_NOMORE
/// A successful status.
constexpr int kEventLoopOk = 0; // AE_OK
/// Read event on the file descriptor.
constexpr int kEventLoopRead = 1; // AE_READABLE
/// Write event on the file descriptor.
constexpr int kEventLoopWrite = 2; // AE_WRITABLE
typedef long long TimerID; // NOLINT
class EventLoop {
public:
// Signature of the handler that will be called when there is a new event
// on the file descriptor that this handler has been registered for.
//
// The arguments are the event flags (read or write).
using FileCallback = std::function<void(int)>;
// This handler will be called when a timer times out. The timer id is
// passed as an argument. The return is the number of milliseconds the timer
// shall be reset to or kEventLoopTimerDone if the timer shall not be
// triggered again.
using TimerCallback = std::function<int(int64_t)>;
EventLoop();
~EventLoop();
/// Add a new file event handler to the event loop.
///
/// \param fd The file descriptor we are listening to.
/// \param events The flags for events we are listening to (read or write).
/// \param callback The callback that will be called when the event happens.
/// \return Returns true if the event handler was added successfully.
bool AddFileEvent(int fd, int events, const FileCallback& callback);
/// Remove a file event handler from the event loop.
///
/// \param fd The file descriptor of the event handler.
void RemoveFileEvent(int fd);
/// Register a handler that will be called after a time slice of
/// "timeout" milliseconds.
///
/// \param timeout The timeout in milliseconds.
/// \param callback The callback for the timeout.
/// \return The ID of the newly created timer.
int64_t AddTimer(int64_t timeout, const TimerCallback& callback);
/// Remove a timer handler from the event loop.
///
/// \param timer_id The ID of the timer that is to be removed.
/// \return The ae.c error code. TODO(pcm): needs to be standardized
int RemoveTimer(int64_t timer_id);
/// \brief Run the event loop.
void Start();
/// \brief Stop the event loop
void Stop();
void Shutdown();
private:
static void FileEventCallback(aeEventLoop* loop, int fd, void* context, int events);
static int TimerEventCallback(aeEventLoop* loop, TimerID timer_id, void* context);
aeEventLoop* loop_;
std::unordered_map<int, std::unique_ptr<FileCallback>> file_callbacks_;
std::unordered_map<int64_t, std::unique_ptr<TimerCallback>> timer_callbacks_;
};
} // namespace plasma
#endif // PLASMA_EVENTS

View file

@ -0,0 +1,175 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/eviction_policy.h"
#include "plasma/plasma_allocator.h"
#include <algorithm>
#include <sstream>
namespace plasma {
void LRUCache::Add(const ObjectID& key, int64_t size) {
auto it = item_map_.find(key);
ARROW_CHECK(it == item_map_.end());
// Note that it is important to use a list so the iterators stay valid.
item_list_.emplace_front(key, size);
item_map_.emplace(key, item_list_.begin());
used_capacity_ += size;
}
int64_t LRUCache::Remove(const ObjectID& key) {
auto it = item_map_.find(key);
if (it == item_map_.end()) {
return -1;
}
int64_t size = it->second->second;
used_capacity_ -= size;
item_list_.erase(it->second);
item_map_.erase(it);
ARROW_CHECK(used_capacity_ >= 0) << DebugString();
return size;
}
void LRUCache::AdjustCapacity(int64_t delta) {
ARROW_LOG(INFO) << "adjusting global lru capacity from " << Capacity() << " to "
<< (Capacity() + delta) << " (max " << OriginalCapacity() << ")";
capacity_ += delta;
ARROW_CHECK(used_capacity_ >= 0) << DebugString();
}
int64_t LRUCache::Capacity() const { return capacity_; }
int64_t LRUCache::OriginalCapacity() const { return original_capacity_; }
int64_t LRUCache::RemainingCapacity() const { return capacity_ - used_capacity_; }
void LRUCache::Foreach(std::function<void(const ObjectID&)> f) {
for (auto& pair : item_list_) {
f(pair.first);
}
}
std::string LRUCache::DebugString() const {
std::stringstream result;
result << "\n(" << name_ << ") capacity: " << Capacity();
result << "\n(" << name_
<< ") used: " << 100. * (1. - (RemainingCapacity() / (double)OriginalCapacity()))
<< "%";
result << "\n(" << name_ << ") num objects: " << item_map_.size();
result << "\n(" << name_ << ") num evictions: " << num_evictions_total_;
result << "\n(" << name_ << ") bytes evicted: " << bytes_evicted_total_;
return result.str();
}
int64_t LRUCache::ChooseObjectsToEvict(int64_t num_bytes_required,
std::vector<ObjectID>* objects_to_evict) {
int64_t bytes_evicted = 0;
auto it = item_list_.end();
while (bytes_evicted < num_bytes_required && it != item_list_.begin()) {
it--;
objects_to_evict->push_back(it->first);
bytes_evicted += it->second;
bytes_evicted_total_ += it->second;
num_evictions_total_ += 1;
}
return bytes_evicted;
}
EvictionPolicy::EvictionPolicy(PlasmaStoreInfo* store_info, int64_t max_size)
: pinned_memory_bytes_(0), store_info_(store_info), cache_("global lru", max_size) {}
int64_t EvictionPolicy::ChooseObjectsToEvict(int64_t num_bytes_required,
std::vector<ObjectID>* objects_to_evict) {
int64_t bytes_evicted =
cache_.ChooseObjectsToEvict(num_bytes_required, objects_to_evict);
// Update the LRU cache.
for (auto& object_id : *objects_to_evict) {
cache_.Remove(object_id);
}
return bytes_evicted;
}
void EvictionPolicy::ObjectCreated(const ObjectID& object_id, Client* client,
bool is_create) {
cache_.Add(object_id, GetObjectSize(object_id));
}
bool EvictionPolicy::SetClientQuota(Client* client, int64_t output_memory_quota) {
return false;
}
bool EvictionPolicy::EnforcePerClientQuota(Client* client, int64_t size, bool is_create,
std::vector<ObjectID>* objects_to_evict) {
return true;
}
void EvictionPolicy::ClientDisconnected(Client* client) {}
bool EvictionPolicy::RequireSpace(int64_t size, std::vector<ObjectID>* objects_to_evict) {
// Check if there is enough space to create the object.
int64_t required_space =
PlasmaAllocator::Allocated() + size - PlasmaAllocator::GetFootprintLimit();
// Try to free up at least as much space as we need right now but ideally
// up to 20% of the total capacity.
int64_t space_to_free =
std::max(required_space, PlasmaAllocator::GetFootprintLimit() / 5);
ARROW_LOG(DEBUG) << "not enough space to create this object, so evicting objects";
// Choose some objects to evict, and update the return pointers.
int64_t num_bytes_evicted = ChooseObjectsToEvict(space_to_free, objects_to_evict);
ARROW_LOG(INFO) << "There is not enough space to create this object, so evicting "
<< objects_to_evict->size() << " objects to free up "
<< num_bytes_evicted << " bytes. The number of bytes in use (before "
<< "this eviction) is " << PlasmaAllocator::Allocated() << ".";
return num_bytes_evicted >= required_space && num_bytes_evicted > 0;
}
void EvictionPolicy::BeginObjectAccess(const ObjectID& object_id) {
// If the object is in the LRU cache, remove it.
cache_.Remove(object_id);
pinned_memory_bytes_ += GetObjectSize(object_id);
}
void EvictionPolicy::EndObjectAccess(const ObjectID& object_id) {
auto size = GetObjectSize(object_id);
// Add the object to the LRU cache.
cache_.Add(object_id, size);
pinned_memory_bytes_ -= size;
}
void EvictionPolicy::RemoveObject(const ObjectID& object_id) {
// If the object is in the LRU cache, remove it.
cache_.Remove(object_id);
}
void EvictionPolicy::RefreshObjects(const std::vector<ObjectID>& object_ids) {
for (const auto& object_id : object_ids) {
int64_t size = cache_.Remove(object_id);
if (size != -1) {
cache_.Add(object_id, size);
}
}
}
int64_t EvictionPolicy::GetObjectSize(const ObjectID& object_id) const {
auto entry = store_info_->objects[object_id].get();
return entry->data_size + entry->metadata_size;
}
std::string EvictionPolicy::DebugString() const { return cache_.DebugString(); }
} // namespace plasma

View file

@ -0,0 +1,212 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_EVICTION_POLICY_H
#define PLASMA_EVICTION_POLICY_H
#include <functional>
#include <list>
#include <string>
#include <unordered_map>
#include <utility>
#include <vector>
#include "plasma/common.h"
#include "plasma/plasma.h"
namespace plasma {
// ==== The eviction policy ====
//
// This file contains declaration for all functions and data structures that
// need to be provided if you want to implement a new eviction algorithm for the
// Plasma store.
//
// It does not implement memory quotas; see quota_aware_policy for that.
class LRUCache {
public:
LRUCache(const std::string& name, int64_t size)
: name_(name),
original_capacity_(size),
capacity_(size),
used_capacity_(0),
num_evictions_total_(0),
bytes_evicted_total_(0) {}
void Add(const ObjectID& key, int64_t size);
int64_t Remove(const ObjectID& key);
int64_t ChooseObjectsToEvict(int64_t num_bytes_required,
std::vector<ObjectID>* objects_to_evict);
int64_t OriginalCapacity() const;
int64_t Capacity() const;
int64_t RemainingCapacity() const;
void AdjustCapacity(int64_t delta);
void Foreach(std::function<void(const ObjectID&)>);
std::string DebugString() const;
private:
/// A doubly-linked list containing the items in the cache and
/// their sizes in LRU order.
typedef std::list<std::pair<ObjectID, int64_t>> ItemList;
ItemList item_list_;
/// A hash table mapping the object ID of an object in the cache to its
/// location in the doubly linked list item_list_.
std::unordered_map<ObjectID, ItemList::iterator> item_map_;
/// The name of this cache, used for debugging purposes only.
const std::string name_;
/// The original (max) capacity of this cache in bytes.
const int64_t original_capacity_;
/// The current capacity, which must be <= the original capacity.
int64_t capacity_;
/// The number of bytes used of the available capacity.
int64_t used_capacity_;
/// The number of objects evicted from this cache.
int64_t num_evictions_total_;
/// The number of bytes evicted from this cache.
int64_t bytes_evicted_total_;
};
/// The eviction policy.
class EvictionPolicy {
public:
/// Construct an eviction policy.
///
/// \param store_info Information about the Plasma store that is exposed
/// to the eviction policy.
/// \param max_size Max size in bytes total of objects to store.
explicit EvictionPolicy(PlasmaStoreInfo* store_info, int64_t max_size);
/// Destroy an eviction policy.
virtual ~EvictionPolicy() {}
/// This method will be called whenever an object is first created in order to
/// add it to the LRU cache. This is done so that the first time, the Plasma
/// store calls begin_object_access, we can remove the object from the LRU
/// cache.
///
/// \param object_id The object ID of the object that was created.
/// \param client The pointer to the client.
/// \param is_create Whether we are creating a new object (vs reading an object).
virtual void ObjectCreated(const ObjectID& object_id, Client* client, bool is_create);
/// Set quota for a client.
///
/// \param client The pointer to the client.
/// \param output_memory_quota Set the quota for this client. This can only be
/// called once per client. This is effectively the equivalent of giving
/// the client its own LRU cache instance. The memory for this is taken
/// out of the capacity of the global LRU cache for the client lifetime.
///
/// \return True if enough space can be reserved for the given client quota.
virtual bool SetClientQuota(Client* client, int64_t output_memory_quota);
/// Determine what objects need to be evicted to enforce the given client's quota.
///
/// \param client The pointer to the client creating the object.
/// \param size The size of the object to create.
/// \param is_create Whether we are creating a new object (vs reading an object).
/// \param objects_to_evict The object IDs that were chosen for eviction will
/// be stored into this vector.
///
/// \return True if enough space could be freed and false otherwise.
virtual bool EnforcePerClientQuota(Client* client, int64_t size, bool is_create,
std::vector<ObjectID>* objects_to_evict);
/// Called to clean up any resources allocated by this client. This merges any
/// per-client LRU queue created by SetClientQuota into the global LRU queue.
///
/// \param client The pointer to the client.
virtual void ClientDisconnected(Client* client);
/// This method will be called when the Plasma store needs more space, perhaps
/// to create a new object. When this method is called, the eviction
/// policy will assume that the objects chosen to be evicted will in fact be
/// evicted from the Plasma store by the caller.
///
/// \param size The size in bytes of the new object, including both data and
/// metadata.
/// \param objects_to_evict The object IDs that were chosen for eviction will
/// be stored into this vector.
/// \return True if enough space can be freed and false otherwise.
virtual bool RequireSpace(int64_t size, std::vector<ObjectID>* objects_to_evict);
/// This method will be called whenever an unused object in the Plasma store
/// starts to be used. When this method is called, the eviction policy will
/// assume that the objects chosen to be evicted will in fact be evicted from
/// the Plasma store by the caller.
///
/// \param object_id The ID of the object that is now being used.
virtual void BeginObjectAccess(const ObjectID& object_id);
/// This method will be called whenever an object in the Plasma store that was
/// being used is no longer being used. When this method is called, the
/// eviction policy will assume that the objects chosen to be evicted will in
/// fact be evicted from the Plasma store by the caller.
///
/// \param object_id The ID of the object that is no longer being used.
virtual void EndObjectAccess(const ObjectID& object_id);
/// Choose some objects to evict from the Plasma store. When this method is
/// called, the eviction policy will assume that the objects chosen to be
/// evicted will in fact be evicted from the Plasma store by the caller.
///
/// @note This method is not part of the API. It is exposed in the header file
/// only for testing.
///
/// \param num_bytes_required The number of bytes of space to try to free up.
/// \param objects_to_evict The object IDs that were chosen for eviction will
/// be stored into this vector.
/// \return The total number of bytes of space chosen to be evicted.
virtual int64_t ChooseObjectsToEvict(int64_t num_bytes_required,
std::vector<ObjectID>* objects_to_evict);
/// This method will be called when an object is going to be removed
///
/// \param object_id The ID of the object that is now being used.
virtual void RemoveObject(const ObjectID& object_id);
virtual void RefreshObjects(const std::vector<ObjectID>& object_ids);
/// Returns debugging information for this eviction policy.
virtual std::string DebugString() const;
protected:
/// Returns the size of the object
int64_t GetObjectSize(const ObjectID& object_id) const;
/// The number of bytes pinned by applications.
int64_t pinned_memory_bytes_;
/// Pointer to the plasma store info.
PlasmaStoreInfo* store_info_;
/// Datastructure for the LRU cache.
LRUCache cache_;
};
} // namespace plasma
#endif // PLASMA_EVICTION_POLICY_H

View file

@ -0,0 +1,63 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include <iostream>
#include <sstream>
#include "arrow/util/memory.h"
#include "plasma/external_store.h"
namespace plasma {
Status ExternalStores::ExtractStoreName(const std::string& endpoint,
std::string* store_name) {
size_t off = endpoint.find_first_of(':');
if (off == std::string::npos) {
return Status::Invalid("Malformed endpoint " + endpoint);
}
*store_name = endpoint.substr(0, off);
return Status::OK();
}
void ExternalStores::RegisterStore(const std::string& store_name,
std::shared_ptr<ExternalStore> store) {
Stores().insert({store_name, store});
}
void ExternalStores::DeregisterStore(const std::string& store_name) {
auto it = Stores().find(store_name);
if (it == Stores().end()) {
return;
}
Stores().erase(it);
}
std::shared_ptr<ExternalStore> ExternalStores::GetStore(const std::string& store_name) {
auto it = Stores().find(store_name);
if (it == Stores().end()) {
return nullptr;
}
return it->second;
}
ExternalStores::StoreMap& ExternalStores::Stores() {
static auto* external_stores = new StoreMap();
return *external_stores;
}
} // namespace plasma

View file

@ -0,0 +1,123 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef EXTERNAL_STORE_H
#define EXTERNAL_STORE_H
#include <memory>
#include <string>
#include <unordered_map>
#include <vector>
#include "plasma/client.h"
namespace plasma {
// ==== The external store ====
//
// This file contains declaration for all functions that need to be implemented
// for an external storage service so that objects evicted from Plasma store
// can be written to it.
class ExternalStore {
public:
/// Default constructor.
ExternalStore() = default;
/// Virtual destructor.
virtual ~ExternalStore() = default;
/// Connect to the local plasma store. Return the resulting connection.
///
/// \param endpoint The name of the endpoint to connect to the external
/// storage service. While the formatting of the endpoint name is
/// specific to the implementation of the external store, it always
/// starts with {store-name}://, where {store-name} is the name of the
/// external store.
///
/// \return The return status.
virtual Status Connect(const std::string& endpoint) = 0;
/// This method will be called whenever an object in the Plasma store needs
/// to be evicted to the external store.
///
/// This API is experimental and might change in the future.
///
/// \param ids The IDs of the objects to put.
/// \param data The object data to put.
/// \return The return status.
virtual Status Put(const std::vector<ObjectID>& ids,
const std::vector<std::shared_ptr<Buffer>>& data) = 0;
/// This method will be called whenever an evicted object in the external
/// store store needs to be accessed.
///
/// This API is experimental and might change in the future.
///
/// \param ids The IDs of the objects to get.
/// \param buffers List of buffers the data should be written to.
/// \return The return status.
virtual Status Get(const std::vector<ObjectID>& ids,
std::vector<std::shared_ptr<Buffer>> buffers) = 0;
};
class ExternalStores {
public:
typedef std::unordered_map<std::string, std::shared_ptr<ExternalStore>> StoreMap;
/// Extracts the external store name from the external store endpoint.
///
/// \param endpoint The endpoint for the external store.
/// \param[out] store_name The name of the external store.
/// \return The return status.
static Status ExtractStoreName(const std::string& endpoint, std::string* store_name);
/// Register a new external store.
///
/// \param store_name Name of the new external store.
/// \param store The new external store object.
static void RegisterStore(const std::string& store_name,
std::shared_ptr<ExternalStore> store);
/// Remove an external store from the registry.
///
/// \param store_name Name of the external store to remove.
static void DeregisterStore(const std::string& store_name);
/// Obtain the external store given its name.
///
/// \param store_name Name of the external store.
/// \return The external store object.
static std::shared_ptr<ExternalStore> GetStore(const std::string& store_name);
private:
/// Obtain mapping between external store names and store instances.
///
/// \return Mapping between external store names and store instances.
static StoreMap& Stores();
};
#define REGISTER_EXTERNAL_STORE(name, store) \
class store##Class { \
public: \
store##Class() { ExternalStores::RegisterStore(name, std::make_shared<store>()); } \
~store##Class() { ExternalStores::DeregisterStore(name); } \
}; \
store##Class singleton_##store = store##Class()
} // namespace plasma
#endif // EXTERNAL_STORE_H

164
src/ray/plasma/fling.cc Normal file
View file

@ -0,0 +1,164 @@
// Copyright 2013 Sharvil Nanavati
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "plasma/fling.h"
#include <string.h>
#include "arrow/util/logging.h"
#ifdef _WIN32
#include <ws2tcpip.h> // socklen_t
#else
typedef int SOCKET;
#endif
void init_msg(struct msghdr* msg, struct iovec* iov, char* buf, size_t buf_len) {
iov->iov_base = buf;
iov->iov_len = 1;
msg->msg_flags = 0;
msg->msg_iov = iov;
msg->msg_iovlen = 1;
msg->msg_control = buf;
msg->msg_controllen = static_cast<socklen_t>(buf_len);
msg->msg_name = NULL;
msg->msg_namelen = 0;
}
int send_fd(int conn, int fd) {
struct msghdr msg;
struct iovec iov;
#ifdef _WIN32
SOCKET to_send = fh_get(fd);
#else
SOCKET to_send = fd;
#endif
char buf[CMSG_SPACE(sizeof(to_send))];
memset(&buf, 0, sizeof(buf));
init_msg(&msg, &iov, buf, sizeof(buf));
struct cmsghdr* header = CMSG_FIRSTHDR(&msg);
if (header == nullptr) {
return -1;
}
header->cmsg_level = SOL_SOCKET;
header->cmsg_type = SCM_RIGHTS;
header->cmsg_len = CMSG_LEN(sizeof(to_send));
memcpy(CMSG_DATA(header), reinterpret_cast<void*>(&to_send), sizeof(to_send));
#ifdef _WIN32
SOCKET sock = fh_get(conn);
#else
SOCKET sock = conn;
#endif
// Send file descriptor.
while (true) {
ssize_t r = sendmsg(sock, &msg, 0);
if (r < 0) {
if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) {
continue;
} else if (errno == EMSGSIZE) {
ARROW_LOG(WARNING) << "Failed to send file descriptor"
<< " (errno = EMSGSIZE), retrying.";
// If we failed to send the file descriptor, loop until we have sent it
// successfully. TODO(rkn): This is problematic for two reasons. First
// of all, sending the file descriptor should just succeed without any
// errors, but sometimes I see a "Message too long" error number.
// Second, looping like this allows a client to potentially block the
// plasma store event loop which should never happen.
continue;
} else {
ARROW_LOG(INFO) << "Error in send_fd (errno = " << errno << ")";
return static_cast<int>(r);
}
} else if (r == 0) {
ARROW_LOG(INFO) << "Encountered unexpected EOF";
return 0;
} else {
ARROW_CHECK(r > 0);
return static_cast<int>(r);
}
}
}
int recv_fd(int conn) {
struct msghdr msg;
struct iovec iov;
char buf[CMSG_SPACE(sizeof(SOCKET))];
init_msg(&msg, &iov, buf, sizeof(buf));
#ifdef _WIN32
SOCKET sock = fh_get(conn);
#else
int sock = conn;
#endif
while (true) {
ssize_t r = recvmsg(sock, &msg, 0);
if (r == -1) {
if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) {
continue;
} else {
ARROW_LOG(INFO) << "Error in recv_fd (errno = " << errno << ")";
return -1;
}
} else {
break;
}
}
SOCKET found_fd = -1;
int oh_noes = 0;
for (struct cmsghdr* header = CMSG_FIRSTHDR(&msg); header != NULL;
header = CMSG_NXTHDR(&msg, header))
if (header->cmsg_level == SOL_SOCKET && header->cmsg_type == SCM_RIGHTS) {
ssize_t count = (header->cmsg_len -
(CMSG_DATA(header) - reinterpret_cast<unsigned char*>(header))) /
sizeof(SOCKET);
for (int i = 0; i < count; ++i) {
SOCKET fd = (reinterpret_cast<SOCKET*>(CMSG_DATA(header)))[i];
if (found_fd == -1) {
found_fd = fd;
} else {
#ifdef _WIN32
closesocket(fd) == 0 || ((WSAGetLastError() == WSAENOTSOCK || WSAGetLastError() == WSANOTINITIALISED) && CloseHandle(reinterpret_cast<HANDLE>(fd)));
#else
close(fd);
#endif
oh_noes = 1;
}
}
}
// The sender sent us more than one file descriptor. We've closed
// them all to prevent fd leaks but notify the caller that we got
// a bad message.
if (oh_noes) {
#ifdef _WIN32
closesocket(found_fd) == 0 || ((WSAGetLastError() == WSAENOTSOCK || WSAGetLastError() == WSANOTINITIALISED) && CloseHandle(reinterpret_cast<HANDLE>(found_fd)));
#else
close(found_fd);
#endif
errno = EBADMSG;
return -1;
}
#ifdef _WIN32
int to_receive = fh_open(found_fd, -1);
#else
int to_receive = found_fd;
#endif
return to_receive;
}

52
src/ray/plasma/fling.h Normal file
View file

@ -0,0 +1,52 @@
// Copyright 2013 Sharvil Nanavati
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// FLING: Exchanging file descriptors over sockets
//
// This is a little library for sending file descriptors over a socket
// between processes. The reason for doing that (as opposed to using
// filenames to share the files) is so (a) no files remain in the
// filesystem after all the processes terminate, (b) to make sure that
// there are no name collisions and (c) to be able to control who has
// access to the data.
//
// Most of the code is from https://github.com/sharvil/flingfd
#include <errno.h>
#include <sys/socket.h>
#include <sys/types.h>
#include <sys/un.h>
#include <unistd.h>
// This is necessary for Mac OS X, see http://www.apuebook.com/faqs2e.html
// (10).
#if !defined(CMSG_SPACE) && !defined(CMSG_LEN)
#define CMSG_SPACE(len) (__DARWIN_ALIGN32(sizeof(struct cmsghdr)) + __DARWIN_ALIGN32(len))
#define CMSG_LEN(len) (__DARWIN_ALIGN32(sizeof(struct cmsghdr)) + (len))
#endif
void init_msg(struct msghdr* msg, struct iovec* iov, char* buf, size_t buf_len);
// Send a file descriptor over a unix domain socket.
//
// \param conn Unix domain socket to send the file descriptor over.
// \param fd File descriptor to send over.
// \return Status code which is < 0 on failure.
int send_fd(int conn, int fd);
// Receive a file descriptor over a unix domain socket.
//
// \param conn Unix domain socket to receive the file descriptor from.
// \return File descriptor or a value < 0 on failure.
int recv_fd(int conn);

View file

@ -0,0 +1,58 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include <memory>
#include <string>
#include "arrow/util/logging.h"
#include "plasma/hash_table_store.h"
namespace plasma {
Status HashTableStore::Connect(const std::string& endpoint) { return Status::OK(); }
Status HashTableStore::Put(const std::vector<ObjectID>& ids,
const std::vector<std::shared_ptr<Buffer>>& data) {
for (size_t i = 0; i < ids.size(); ++i) {
table_[ids[i]] = data[i]->ToString();
}
return Status::OK();
}
Status HashTableStore::Get(const std::vector<ObjectID>& ids,
std::vector<std::shared_ptr<Buffer>> buffers) {
ARROW_CHECK(ids.size() == buffers.size());
for (size_t i = 0; i < ids.size(); ++i) {
bool valid;
HashTable::iterator result;
{
result = table_.find(ids[i]);
valid = result != table_.end();
}
if (valid) {
ARROW_CHECK(buffers[i]->size() == static_cast<int64_t>(result->second.size()));
std::memcpy(buffers[i]->mutable_data(), result->second.data(),
result->second.size());
}
}
return Status::OK();
}
REGISTER_EXTERNAL_STORE("hashtable", HashTableStore);
} // namespace plasma

View file

@ -0,0 +1,53 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef HASH_TABLE_STORE_H
#define HASH_TABLE_STORE_H
#include <memory>
#include <string>
#include <unordered_map>
#include <vector>
#include "plasma/external_store.h"
namespace plasma {
// This is a sample implementation for an external store, for illustration
// purposes only.
class HashTableStore : public ExternalStore {
public:
HashTableStore() = default;
Status Connect(const std::string& endpoint) override;
Status Get(const std::vector<ObjectID>& ids,
std::vector<std::shared_ptr<Buffer>> buffers) override;
Status Put(const std::vector<ObjectID>& ids,
const std::vector<std::shared_ptr<Buffer>>& data) override;
private:
typedef std::unordered_map<ObjectID, std::string> HashTable;
HashTable table_;
};
} // namespace plasma
#endif // HASH_TABLE_STORE_H

257
src/ray/plasma/io.cc Normal file
View file

@ -0,0 +1,257 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/io.h"
#include <cstdint>
#include <memory>
#include <sstream>
#include "arrow/status.h"
#include "arrow/util/logging.h"
#include "plasma/common.h"
#include "plasma/plasma_generated.h"
#ifndef _WIN32
#include <arpa/inet.h>
#include <netinet/in.h>
#endif
using arrow::Status;
/// Number of times we try connecting to a socket.
constexpr int64_t kNumConnectAttempts = 80;
/// Time to wait between connection attempts to a socket.
constexpr int64_t kConnectTimeoutMs = 100;
namespace plasma {
using flatbuf::MessageType;
Status WriteBytes(int fd, uint8_t* cursor, size_t length) {
ssize_t nbytes = 0;
size_t bytesleft = length;
size_t offset = 0;
while (bytesleft > 0) {
// While we haven't written the whole message, write to the file descriptor,
// advance the cursor, and decrease the amount left to write.
nbytes = write(fd, cursor + offset, bytesleft);
if (nbytes < 0) {
if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) {
continue;
}
return Status::IOError(strerror(errno));
} else if (nbytes == 0) {
return Status::IOError("Encountered unexpected EOF");
}
ARROW_CHECK(nbytes > 0);
bytesleft -= nbytes;
offset += nbytes;
}
return Status::OK();
}
Status WriteMessage(int fd, MessageType type, int64_t length, uint8_t* bytes) {
int64_t version = kPlasmaProtocolVersion;
RETURN_NOT_OK(WriteBytes(fd, reinterpret_cast<uint8_t*>(&version), sizeof(version)));
RETURN_NOT_OK(WriteBytes(fd, reinterpret_cast<uint8_t*>(&type), sizeof(type)));
RETURN_NOT_OK(WriteBytes(fd, reinterpret_cast<uint8_t*>(&length), sizeof(length)));
return WriteBytes(fd, bytes, length * sizeof(char));
}
Status ReadBytes(int fd, uint8_t* cursor, size_t length) {
ssize_t nbytes = 0;
// Termination condition: EOF or read 'length' bytes total.
size_t bytesleft = length;
size_t offset = 0;
while (bytesleft > 0) {
nbytes = read(fd, cursor + offset, bytesleft);
if (nbytes < 0) {
if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) {
continue;
}
return Status::IOError(strerror(errno));
} else if (0 == nbytes) {
return Status::IOError("Encountered unexpected EOF");
}
ARROW_CHECK(nbytes > 0);
bytesleft -= nbytes;
offset += nbytes;
}
return Status::OK();
}
Status ReadMessage(int fd, MessageType* type, std::vector<uint8_t>* buffer) {
int64_t version;
RETURN_NOT_OK_ELSE(ReadBytes(fd, reinterpret_cast<uint8_t*>(&version), sizeof(version)),
*type = MessageType::PlasmaDisconnectClient);
ARROW_CHECK(version == kPlasmaProtocolVersion) << "version = " << version;
RETURN_NOT_OK_ELSE(ReadBytes(fd, reinterpret_cast<uint8_t*>(type), sizeof(*type)),
*type = MessageType::PlasmaDisconnectClient);
int64_t length_temp;
RETURN_NOT_OK_ELSE(
ReadBytes(fd, reinterpret_cast<uint8_t*>(&length_temp), sizeof(length_temp)),
*type = MessageType::PlasmaDisconnectClient);
// The length must be read as an int64_t, but it should be used as a size_t.
size_t length = static_cast<size_t>(length_temp);
if (length > buffer->size()) {
buffer->resize(length);
}
RETURN_NOT_OK_ELSE(ReadBytes(fd, buffer->data(), length),
*type = MessageType::PlasmaDisconnectClient);
return Status::OK();
}
int ConnectOrListenIpcSock(const std::string& pathname, bool shall_listen) {
union {
struct sockaddr addr;
struct sockaddr_un un;
struct sockaddr_in in;
} socket_address;
int addrlen;
memset(&socket_address, 0, sizeof(socket_address));
if (pathname.find("tcp://") == 0) {
addrlen = sizeof(socket_address.in);
socket_address.in.sin_family = AF_INET;
std::string addr = pathname.substr(pathname.find('/') + 2);
size_t i = addr.rfind(':'), j;
if (i >= addr.size()) {
j = i = addr.size();
} else {
j = i + 1;
}
socket_address.in.sin_addr.s_addr = inet_addr(addr.substr(0, i).c_str());
socket_address.in.sin_port = htons(static_cast<short>(atoi(addr.substr(j).c_str())));
if (socket_address.in.sin_addr.s_addr == INADDR_NONE) {
ARROW_LOG(ERROR) << "Socket address is not a valid IPv4 address: " << pathname;
return -1;
}
if (socket_address.in.sin_port == htons(0)) {
ARROW_LOG(ERROR) << "Socket address is missing a valid port: " << pathname;
return -1;
}
} else {
addrlen = sizeof(socket_address.un);
socket_address.un.sun_family = AF_UNIX;
if (pathname.size() + 1 > sizeof(socket_address.un.sun_path)) {
ARROW_LOG(ERROR) << "Socket pathname is too long.";
return -1;
}
strncpy(socket_address.un.sun_path, pathname.c_str(), pathname.size() + 1);
}
int socket_fd = socket(socket_address.addr.sa_family, SOCK_STREAM, 0);
if (socket_fd < 0) {
ARROW_LOG(ERROR) << "socket() failed for pathname " << pathname;
return -1;
}
if (shall_listen) {
// Tell the system to allow the port to be reused.
int on = 1;
if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast<char*>(&on),
sizeof(on)) < 0) {
ARROW_LOG(ERROR) << "setsockopt failed for pathname " << pathname;
close(socket_fd);
return -1;
}
if (socket_address.addr.sa_family == AF_UNIX) {
#ifdef _WIN32
_unlink(pathname.c_str());
#else
unlink(pathname.c_str());
#endif
}
if (bind(socket_fd, &socket_address.addr, addrlen) != 0) {
ARROW_LOG(ERROR) << "Bind failed for pathname " << pathname;
close(socket_fd);
return -1;
}
if (listen(socket_fd, 128) == -1) {
ARROW_LOG(ERROR) << "Could not listen to socket " << pathname;
close(socket_fd);
return -1;
}
} else {
if (connect(socket_fd, &socket_address.addr, addrlen) != 0) {
close(socket_fd);
return -1;
}
}
return socket_fd;
}
Status ConnectIpcSocketRetry(const std::string& pathname, int num_retries,
int64_t timeout, int* fd) {
// Pick the default values if the user did not specify.
if (num_retries < 0) {
num_retries = kNumConnectAttempts;
}
if (timeout < 0) {
timeout = kConnectTimeoutMs;
}
*fd = ConnectOrListenIpcSock(pathname, false);
while (*fd < 0 && num_retries > 0) {
ARROW_LOG(ERROR) << "Connection to IPC socket failed for pathname " << pathname
<< ", retrying " << num_retries << " more times";
// Sleep for timeout milliseconds.
usleep(static_cast<int>(timeout * 1000));
*fd = ConnectOrListenIpcSock(pathname, false);
--num_retries;
}
// If we could not connect to the socket, exit.
if (*fd == -1) {
return Status::IOError("Could not connect to socket ", pathname);
}
return Status::OK();
}
int AcceptClient(int socket_fd) {
int client_fd = accept(socket_fd, NULL, NULL);
if (client_fd < 0) {
ARROW_LOG(ERROR) << "Error reading from socket.";
return -1;
}
return client_fd;
}
std::unique_ptr<uint8_t[]> ReadMessageAsync(int sock) {
int64_t size;
Status s = ReadBytes(sock, reinterpret_cast<uint8_t*>(&size), sizeof(int64_t));
if (!s.ok()) {
// The other side has closed the socket.
ARROW_LOG(DEBUG) << "Socket has been closed, or some other error has occurred.";
close(sock);
return NULL;
}
auto message = std::unique_ptr<uint8_t[]>(new uint8_t[size]);
s = ReadBytes(sock, message.get(), size);
if (!s.ok()) {
// The other side has closed the socket.
ARROW_LOG(DEBUG) << "Socket has been closed, or some other error has occurred.";
close(sock);
return NULL;
}
return message;
}
} // namespace plasma

69
src/ray/plasma/io.h Normal file
View file

@ -0,0 +1,69 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_IO_H
#define PLASMA_IO_H
#include <inttypes.h>
#include <sys/socket.h>
#include <sys/un.h>
#include <unistd.h>
#include <memory>
#include <string>
#include <vector>
#include "arrow/status.h"
#include "plasma/common.h"
#include "plasma/compat.h"
namespace plasma {
namespace flatbuf {
// Forward declaration outside the namespace, which is defined in plasma_generated.h.
enum class MessageType : int64_t;
} // namespace flatbuf
// TODO(pcm): Replace our own custom message header (message type,
// message length, plasma protocol version) with one that is serialized
// using flatbuffers.
constexpr int64_t kPlasmaProtocolVersion = 0x0000000000000000;
using arrow::Status;
Status WriteBytes(int fd, uint8_t* cursor, size_t length);
Status WriteMessage(int fd, flatbuf::MessageType type, int64_t length, uint8_t* bytes);
Status ReadBytes(int fd, uint8_t* cursor, size_t length);
Status ReadMessage(int fd, flatbuf::MessageType* type, std::vector<uint8_t>* buffer);
int ConnectOrListenIpcSock(const std::string& pathname, bool shall_listen);
Status ConnectIpcSocketRetry(const std::string& pathname, int num_retries,
int64_t timeout, int* fd);
int AcceptClient(int socket_fd);
std::unique_ptr<uint8_t[]> ReadMessageAsync(int sock);
} // namespace plasma
#endif // PLASMA_IO_H

View file

@ -0,0 +1,243 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/lib/java/org_apache_arrow_plasma_PlasmaClientJNI.h"
#include <pthread.h>
#include <stdlib.h>
#include <unistd.h>
#include <algorithm>
#include <cstring>
#include <iostream>
#include <memory>
#include <string>
#include <vector>
#include "arrow/util/logging.h"
#include "plasma/client.h"
constexpr jsize OBJECT_ID_SIZE = sizeof(plasma::ObjectID) / sizeof(jbyte);
inline void jbyteArray_to_object_id(JNIEnv* env, jbyteArray a, plasma::ObjectID* oid) {
env->GetByteArrayRegion(a, 0, OBJECT_ID_SIZE, reinterpret_cast<jbyte*>(oid));
}
inline void object_id_to_jbyteArray(JNIEnv* env, jbyteArray a, plasma::ObjectID* oid) {
env->SetByteArrayRegion(a, 0, OBJECT_ID_SIZE, reinterpret_cast<jbyte*>(oid));
}
inline void throw_exception_if_not_OK(JNIEnv* env, const arrow::Status& status) {
if (!status.ok()) {
jclass Exception =
env->FindClass("org/apache/arrow/plasma/exceptions/PlasmaClientException");
env->ThrowNew(Exception, status.message().c_str());
}
}
class JByteArrayGetter {
private:
JNIEnv* _env;
jbyteArray _a;
jbyte* bp;
public:
JByteArrayGetter(JNIEnv* env, jbyteArray a, jbyte** out) {
_env = env;
_a = a;
bp = _env->GetByteArrayElements(_a, nullptr);
*out = bp;
}
~JByteArrayGetter() { _env->ReleaseByteArrayElements(_a, bp, 0); }
};
JNIEXPORT jlong JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_connect(
JNIEnv* env, jclass cls, jstring store_socket_name, jstring manager_socket_name,
jint release_delay) {
const char* s_name = env->GetStringUTFChars(store_socket_name, nullptr);
const char* m_name = env->GetStringUTFChars(manager_socket_name, nullptr);
plasma::PlasmaClient* client = new plasma::PlasmaClient();
throw_exception_if_not_OK(env, client->Connect(s_name, m_name, release_delay));
env->ReleaseStringUTFChars(store_socket_name, s_name);
env->ReleaseStringUTFChars(manager_socket_name, m_name);
return reinterpret_cast<int64_t>(client);
}
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_disconnect(
JNIEnv* env, jclass cls, jlong conn) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
throw_exception_if_not_OK(env, client->Disconnect());
delete client;
return;
}
JNIEXPORT jobject JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_create(
JNIEnv* env, jclass cls, jlong conn, jbyteArray object_id, jint size,
jbyteArray metadata) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
plasma::ObjectID oid;
jbyteArray_to_object_id(env, object_id, &oid);
// prepare metadata buffer
uint8_t* md = nullptr;
jsize md_size = 0;
std::unique_ptr<JByteArrayGetter> md_getter;
if (metadata != nullptr) {
md_size = env->GetArrayLength(metadata);
}
if (md_size > 0) {
md_getter.reset(new JByteArrayGetter(env, metadata, reinterpret_cast<jbyte**>(&md)));
}
std::shared_ptr<Buffer> data;
Status s = client->Create(oid, size, md, md_size, &data);
if (plasma::IsPlasmaObjectExists(s)) {
jclass exceptionClass =
env->FindClass("org/apache/arrow/plasma/exceptions/DuplicateObjectException");
env->ThrowNew(exceptionClass, oid.hex().c_str());
return nullptr;
}
if (plasma::IsPlasmaStoreFull(s)) {
jclass exceptionClass =
env->FindClass("org/apache/arrow/plasma/exceptions/PlasmaOutOfMemoryException");
env->ThrowNew(exceptionClass, "");
return nullptr;
}
throw_exception_if_not_OK(env, s);
return env->NewDirectByteBuffer(data->mutable_data(), size);
}
JNIEXPORT jbyteArray JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_hash(
JNIEnv* env, jclass cls, jlong conn, jbyteArray object_id) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
plasma::ObjectID oid;
jbyteArray_to_object_id(env, object_id, &oid);
unsigned char digest[plasma::kDigestSize];
bool success = client->Hash(oid, digest).ok();
if (success) {
jbyteArray ret = env->NewByteArray(plasma::kDigestSize);
env->SetByteArrayRegion(ret, 0, plasma::kDigestSize,
reinterpret_cast<jbyte*>(digest));
return ret;
} else {
return nullptr;
}
}
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_seal(
JNIEnv* env, jclass cls, jlong conn, jbyteArray object_id) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
plasma::ObjectID oid;
jbyteArray_to_object_id(env, object_id, &oid);
throw_exception_if_not_OK(env, client->Seal(oid));
}
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_release(
JNIEnv* env, jclass cls, jlong conn, jbyteArray object_id) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
plasma::ObjectID oid;
jbyteArray_to_object_id(env, object_id, &oid);
throw_exception_if_not_OK(env, client->Release(oid));
}
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_delete(
JNIEnv* env, jclass cls, jlong conn, jbyteArray object_id) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
plasma::ObjectID oid;
jbyteArray_to_object_id(env, object_id, &oid);
throw_exception_if_not_OK(env, client->Delete(oid));
}
JNIEXPORT jobjectArray JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_get(
JNIEnv* env, jclass cls, jlong conn, jobjectArray object_ids, jint timeout_ms) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
jsize num_oids = env->GetArrayLength(object_ids);
std::vector<plasma::ObjectID> oids(num_oids);
std::vector<plasma::ObjectBuffer> obufs(num_oids);
for (int i = 0; i < num_oids; ++i) {
jbyteArray_to_object_id(
env, reinterpret_cast<jbyteArray>(env->GetObjectArrayElement(object_ids, i)),
&oids[i]);
}
// TODO: may be blocked. consider to add the thread support
throw_exception_if_not_OK(env,
client->Get(oids.data(), num_oids, timeout_ms, obufs.data()));
jclass clsByteBuffer = env->FindClass("java/nio/ByteBuffer");
jclass clsByteBufferArray = env->FindClass("[Ljava/nio/ByteBuffer;");
jobjectArray ret = env->NewObjectArray(num_oids, clsByteBufferArray, nullptr);
jobjectArray o = nullptr;
jobject dataBuf, metadataBuf;
for (int i = 0; i < num_oids; ++i) {
o = env->NewObjectArray(2, clsByteBuffer, nullptr);
if (obufs[i].data && obufs[i].data->size() != -1) {
dataBuf = env->NewDirectByteBuffer(const_cast<uint8_t*>(obufs[i].data->data()),
obufs[i].data->size());
if (obufs[i].metadata && obufs[i].metadata->size() > 0) {
metadataBuf = env->NewDirectByteBuffer(
const_cast<uint8_t*>(obufs[i].metadata->data()), obufs[i].metadata->size());
} else {
metadataBuf = nullptr;
}
} else {
dataBuf = nullptr;
metadataBuf = nullptr;
}
env->SetObjectArrayElement(o, 0, dataBuf);
env->SetObjectArrayElement(o, 1, metadataBuf);
env->SetObjectArrayElement(ret, i, o);
}
return ret;
}
JNIEXPORT jboolean JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_contains(
JNIEnv* env, jclass cls, jlong conn, jbyteArray object_id) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
plasma::ObjectID oid;
jbyteArray_to_object_id(env, object_id, &oid);
bool has_object;
throw_exception_if_not_OK(env, client->Contains(oid, &has_object));
return has_object;
}
JNIEXPORT jlong JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_evict(
JNIEnv* env, jclass cls, jlong conn, jlong num_bytes) {
plasma::PlasmaClient* client = reinterpret_cast<plasma::PlasmaClient*>(conn);
int64_t evicted_bytes;
throw_exception_if_not_OK(
env, client->Evict(static_cast<int64_t>(num_bytes), evicted_bytes));
return static_cast<jlong>(evicted_bytes);
}

View file

@ -0,0 +1,132 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
/* DO NOT EDIT THIS FILE - it is machine generated */
#include "jni.h"
/* Header for class org_apache_arrow_plasma_PlasmaClientJNI */
#ifndef _Included_org_apache_arrow_plasma_PlasmaClientJNI
#define _Included_org_apache_arrow_plasma_PlasmaClientJNI
#ifdef __cplusplus
extern "C" {
#endif
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: connect
* Signature: (Ljava/lang/String;Ljava/lang/String;I)J
*/
JNIEXPORT jlong JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_connect(
JNIEnv*, jclass, jstring, jstring, jint);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: disconnect
* Signature: (J)V
*/
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_disconnect(JNIEnv*,
jclass,
jlong);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: create
* Signature: (J[BI[B)Ljava/nio/ByteBuffer;
*/
JNIEXPORT jobject JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_create(
JNIEnv*, jclass, jlong, jbyteArray, jint, jbyteArray);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: hash
* Signature: (J[B)[B
*/
JNIEXPORT jbyteArray JNICALL
Java_org_apache_arrow_plasma_PlasmaClientJNI_hash(JNIEnv*, jclass, jlong, jbyteArray);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: seal
* Signature: (J[B)V
*/
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_seal(JNIEnv*, jclass,
jlong,
jbyteArray);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: release
* Signature: (J[B)V
*/
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_release(JNIEnv*,
jclass, jlong,
jbyteArray);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: delete
* Signature: (J[B)V
*/
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_delete(JNIEnv*,
jclass, jlong,
jbyteArray);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: get
* Signature: (J[[BI)[[Ljava/nio/ByteBuffer;
*/
JNIEXPORT jobjectArray JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_get(
JNIEnv*, jclass, jlong, jobjectArray, jint);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: contains
* Signature: (J[B)Z
*/
JNIEXPORT jboolean JNICALL
Java_org_apache_arrow_plasma_PlasmaClientJNI_contains(JNIEnv*, jclass, jlong, jbyteArray);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: fetch
* Signature: (J[[B)V
*/
JNIEXPORT void JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_fetch(JNIEnv*, jclass,
jlong,
jobjectArray);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: wait
* Signature: (J[[BII)[[B
*/
JNIEXPORT jobjectArray JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_wait(
JNIEnv*, jclass, jlong, jobjectArray, jint, jint);
/*
* Class: org_apache_arrow_plasma_PlasmaClientJNI
* Method: evict
* Signature: (JJ)J
*/
JNIEXPORT jlong JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_evict(JNIEnv*,
jclass, jlong,
jlong);
#ifdef __cplusplus
}
#endif
#endif

68
src/ray/plasma/malloc.cc Normal file
View file

@ -0,0 +1,68 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/malloc.h"
#include <assert.h>
#include <stddef.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <cerrno>
#include <string>
#include <vector>
#include "plasma/common.h"
#include "plasma/plasma.h"
namespace plasma {
std::unordered_map<void*, MmapRecord> mmap_records;
static void* pointer_advance(void* p, ptrdiff_t n) { return (unsigned char*)p + n; }
static ptrdiff_t pointer_distance(void const* pfrom, void const* pto) {
return (unsigned char const*)pto - (unsigned char const*)pfrom;
}
void GetMallocMapinfo(void* addr, int* fd, int64_t* map_size, ptrdiff_t* offset) {
// TODO(rshin): Implement a more efficient search through mmap_records.
for (const auto& entry : mmap_records) {
if (addr >= entry.first && addr < pointer_advance(entry.first, entry.second.size)) {
*fd = entry.second.fd;
*map_size = entry.second.size;
*offset = pointer_distance(entry.first, addr);
return;
}
}
*fd = -1;
*map_size = 0;
*offset = 0;
}
int64_t GetMmapSize(int fd) {
for (const auto& entry : mmap_records) {
if (entry.second.fd == fd) {
return entry.second.size;
}
}
ARROW_LOG(FATAL) << "failed to find entry in mmap_records for fd " << fd;
return -1; // This code is never reached.
}
} // namespace plasma

54
src/ray/plasma/malloc.h Normal file
View file

@ -0,0 +1,54 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_MALLOC_H
#define PLASMA_MALLOC_H
#include <inttypes.h>
#include <stddef.h>
#include <unordered_map>
namespace plasma {
/// Gap between two consecutive mmap regions allocated by fake_mmap.
/// This ensures that the segments of memory returned by
/// fake_mmap are never contiguous and dlmalloc does not coalesce it
/// (in the client we cannot guarantee that these mmaps are contiguous).
constexpr int64_t kMmapRegionsGap = sizeof(size_t);
void GetMallocMapinfo(void* addr, int* fd, int64_t* map_length, ptrdiff_t* offset);
/// Get the mmap size corresponding to a specific file descriptor.
///
/// \param fd The file descriptor to look up.
/// \return The size of the corresponding memory-mapped file.
int64_t GetMmapSize(int fd);
struct MmapRecord {
int fd;
int64_t size;
};
/// Hashtable that contains one entry per segment that we got from the OS
/// via mmap. Associates the address of that segment with its file descriptor
/// and size.
extern std::unordered_map<void*, MmapRecord> mmap_records;
} // namespace plasma
#endif // PLASMA_MALLOC_H

99
src/ray/plasma/plasma.cc Normal file
View file

@ -0,0 +1,99 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/plasma.h"
#include <sys/socket.h>
#include <sys/types.h>
#include <unistd.h>
#include "plasma/common.h"
#include "plasma/common_generated.h"
#include "plasma/protocol.h"
namespace fb = plasma::flatbuf;
namespace plasma {
ObjectTableEntry::ObjectTableEntry() : pointer(nullptr), ref_count(0) {}
ObjectTableEntry::~ObjectTableEntry() { pointer = nullptr; }
int WarnIfSigpipe(int status, int client_sock) {
if (status >= 0) {
return 0;
}
if (errno == EPIPE || errno == EBADF || errno == ECONNRESET) {
ARROW_LOG(WARNING) << "Received SIGPIPE, BAD FILE DESCRIPTOR, or ECONNRESET when "
"sending a message to client on fd "
<< client_sock
<< ". The client on the other end may "
"have hung up.";
return errno;
}
ARROW_LOG(FATAL) << "Failed to write message to client on fd " << client_sock << ".";
return -1; // This is never reached.
}
/**
* This will create a new ObjectInfo buffer. The first sizeof(int64_t) bytes
* of this buffer are the length of the remaining message and the
* remaining message is a serialized version of the object info.
*
* \param object_info The object info to be serialized
* \return The object info buffer. It is the caller's responsibility to free
* this buffer with "delete" after it has been used.
*/
std::unique_ptr<uint8_t[]> CreateObjectInfoBuffer(fb::ObjectInfoT* object_info) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreateObjectInfo(fbb, object_info);
fbb.Finish(message);
auto notification =
std::unique_ptr<uint8_t[]>(new uint8_t[sizeof(int64_t) + fbb.GetSize()]);
*(reinterpret_cast<int64_t*>(notification.get())) = fbb.GetSize();
memcpy(notification.get() + sizeof(int64_t), fbb.GetBufferPointer(), fbb.GetSize());
return notification;
}
std::unique_ptr<uint8_t[]> CreatePlasmaNotificationBuffer(
std::vector<fb::ObjectInfoT>& object_info) {
flatbuffers::FlatBufferBuilder fbb;
std::vector<flatbuffers::Offset<plasma::flatbuf::ObjectInfo>> info;
for (size_t i = 0; i < object_info.size(); ++i) {
info.push_back(fb::CreateObjectInfo(fbb, &object_info[i]));
}
auto info_array = fbb.CreateVector(info);
auto message = fb::CreatePlasmaNotification(fbb, info_array);
fbb.Finish(message);
auto notification =
std::unique_ptr<uint8_t[]>(new uint8_t[sizeof(int64_t) + fbb.GetSize()]);
*(reinterpret_cast<int64_t*>(notification.get())) = fbb.GetSize();
memcpy(notification.get() + sizeof(int64_t), fbb.GetBufferPointer(), fbb.GetSize());
return notification;
}
ObjectTableEntry* GetObjectTableEntry(PlasmaStoreInfo* store_info,
const ObjectID& object_id) {
auto it = store_info->objects.find(object_id);
if (it == store_info->objects.end()) {
return NULL;
}
return it->second.get();
}
} // namespace plasma

357
src/ray/plasma/plasma.fbs Normal file
View file

@ -0,0 +1,357 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
include "common.fbs";
// Plasma protocol specification
namespace plasma.flatbuf;
enum MessageType:long {
// Message that gets send when a client hangs up.
PlasmaDisconnectClient = 0,
// Create a new object.
PlasmaCreateRequest,
PlasmaCreateReply,
PlasmaCreateAndSealRequest,
PlasmaCreateAndSealReply,
PlasmaAbortRequest,
PlasmaAbortReply,
// Seal an object.
PlasmaSealRequest,
PlasmaSealReply,
// Get an object that is stored on the local Plasma store.
PlasmaGetRequest,
PlasmaGetReply,
// Release an object.
PlasmaReleaseRequest,
PlasmaReleaseReply,
// Delete an object.
PlasmaDeleteRequest,
PlasmaDeleteReply,
// See if the store contains an object (will be deprecated).
PlasmaContainsRequest,
PlasmaContainsReply,
// List all objects in the store.
PlasmaListRequest,
PlasmaListReply,
// Get information for a newly connecting client.
PlasmaConnectRequest,
PlasmaConnectReply,
// Make room for new objects in the plasma store.
PlasmaEvictRequest,
PlasmaEvictReply,
// Subscribe to a list of objects or to all objects.
PlasmaSubscribeRequest,
// Unsubscribe.
PlasmaUnsubscribeRequest,
// Sending and receiving data.
// PlasmaDataRequest initiates sending the data, there will be one
// such message per data transfer.
PlasmaDataRequest,
// PlasmaDataReply contains the actual data and is sent back to the
// object store that requested the data. For each transfer, multiple
// reply messages get sent. Each one contains a fixed number of bytes.
PlasmaDataReply,
// Object notifications.
PlasmaNotification,
// Set memory quota for a client.
PlasmaSetOptionsRequest,
PlasmaSetOptionsReply,
// Get debugging information from the store.
PlasmaGetDebugStringRequest,
PlasmaGetDebugStringReply,
// Create and seal a batch of objects. This should be used to save
// IPC for creating many small objects.
PlasmaCreateAndSealBatchRequest,
PlasmaCreateAndSealBatchReply,
// Touch a number of objects to bump their position in the LRU cache.
PlasmaRefreshLRURequest,
PlasmaRefreshLRUReply,
}
enum PlasmaError:int {
// Operation was successful.
OK,
// Trying to create an object that already exists.
ObjectExists,
// Trying to access an object that doesn't exist.
ObjectNonexistent,
// Trying to create an object but there isn't enough space in the store.
OutOfMemory,
// Trying to delete an object but it's not sealed.
ObjectNotSealed,
// Trying to delete an object but it's in use.
ObjectInUse,
}
// Plasma store messages
struct PlasmaObjectSpec {
// Index of the memory segment (= memory mapped file) that
// this object is allocated in.
segment_index: int;
// The offset in bytes in the memory mapped file of the data.
data_offset: ulong;
// The size in bytes of the data.
data_size: ulong;
// The offset in bytes in the memory mapped file of the metadata.
metadata_offset: ulong;
// The size in bytes of the metadata.
metadata_size: ulong;
// Device to create buffer on.
device_num: int;
}
table PlasmaSetOptionsRequest {
// The name of the client.
client_name: string;
// The size of the output memory limit in bytes.
output_memory_quota: long;
}
table PlasmaSetOptionsReply {
// Whether setting options succeeded.
error: PlasmaError;
}
table PlasmaGetDebugStringRequest {
}
table PlasmaGetDebugStringReply {
// The debug string from the server.
debug_string: string;
}
table PlasmaCreateRequest {
// ID of the object to be created.
object_id: string;
// Whether to evict other objects to make room for this one.
evict_if_full: bool;
// The size of the object's data in bytes.
data_size: ulong;
// The size of the object's metadata in bytes.
metadata_size: ulong;
// Device to create buffer on.
device_num: int;
}
table CudaHandle {
handle: [ubyte];
}
table PlasmaCreateReply {
// ID of the object that was created.
object_id: string;
// The object that is returned with this reply.
plasma_object: PlasmaObjectSpec;
// Error that occurred for this call.
error: PlasmaError;
// The file descriptor in the store that corresponds to the file descriptor
// being sent to the client right after this message.
store_fd: int;
// The size in bytes of the segment for the store file descriptor (needed to
// call mmap).
mmap_size: long;
// CUDA IPC Handle for objects on GPU.
ipc_handle: CudaHandle;
}
table PlasmaCreateAndSealRequest {
// ID of the object to be created.
object_id: string;
// Whether to evict other objects to make room for this one.
evict_if_full: bool;
// The object's data.
data: string;
// The object's metadata.
metadata: string;
// Hash of the object data.
digest: string;
}
table PlasmaCreateAndSealReply {
// Error that occurred for this call.
error: PlasmaError;
}
table PlasmaCreateAndSealBatchRequest {
object_ids: [string];
// Whether to evict other objects to make room for these objects.
evict_if_full: bool;
data: [string];
metadata: [string];
digest: [string];
}
table PlasmaCreateAndSealBatchReply {
// Error that occurred for this call.
error: PlasmaError;
}
table PlasmaAbortRequest {
// ID of the object to be aborted.
object_id: string;
}
table PlasmaAbortReply {
// ID of the object that was aborted.
object_id: string;
}
table PlasmaSealRequest {
// ID of the object to be sealed.
object_id: string;
// Hash of the object data.
digest: string;
}
table PlasmaSealReply {
// ID of the object that was sealed.
object_id: string;
// Error code.
error: PlasmaError;
}
table PlasmaGetRequest {
// IDs of the objects stored at local Plasma store we are getting.
object_ids: [string];
// The number of milliseconds before the request should timeout.
timeout_ms: long;
}
table PlasmaGetReply {
// IDs of the objects being returned.
// This number can be smaller than the number of requested
// objects if not all requested objects are stored and sealed
// in the local Plasma store.
object_ids: [string];
// Plasma object information, in the same order as their IDs. The number of
// elements in both object_ids and plasma_objects arrays must agree.
plasma_objects: [PlasmaObjectSpec];
// A list of the file descriptors in the store that correspond to the file
// descriptors being sent to the client. The length of this list is the number
// of file descriptors that the store will send to the client after this
// message.
store_fds: [int];
// Size in bytes of the segment for each store file descriptor (needed to call
// mmap). This list must have the same length as store_fds.
mmap_sizes: [long];
// The number of elements in both object_ids and plasma_objects arrays must agree.
handles: [CudaHandle];
}
table PlasmaReleaseRequest {
// ID of the object to be released.
object_id: string;
}
table PlasmaReleaseReply {
// ID of the object that was released.
object_id: string;
// Error code.
error: PlasmaError;
}
table PlasmaDeleteRequest {
// The number of objects to delete.
count: int;
// ID of the object to be deleted.
object_ids: [string];
}
table PlasmaDeleteReply {
// The number of objects to delete.
count: int;
// ID of the object that was deleted.
object_ids: [string];
// Error code.
errors: [PlasmaError];
}
table PlasmaContainsRequest {
// ID of the object we are querying.
object_id: string;
}
table PlasmaContainsReply {
// ID of the object we are querying.
object_id: string;
// 1 if the object is in the store and 0 otherwise.
has_object: int;
}
table PlasmaListRequest {
}
table PlasmaListReply {
objects: [ObjectInfo];
}
// PlasmaConnect is used by a plasma client the first time it connects with the
// store. This is not really necessary, but is used to get some information
// about the store such as its memory capacity.
table PlasmaConnectRequest {
}
table PlasmaConnectReply {
// The memory capacity of the store.
memory_capacity: long;
}
table PlasmaEvictRequest {
// Number of bytes that shall be freed.
num_bytes: ulong;
}
table PlasmaEvictReply {
// Number of bytes that have been freed.
num_bytes: ulong;
}
table PlasmaSubscribeRequest {
}
table PlasmaNotification {
object_info: [ObjectInfo];
}
table PlasmaDataRequest {
// ID of the object that is requested.
object_id: string;
// The host address where the data shall be sent to.
address: string;
// The port of the manager the data shall be sent to.
port: int;
}
table PlasmaDataReply {
// ID of the object that will be sent.
object_id: string;
// Size of the object data in bytes.
object_size: ulong;
// Size of the metadata in bytes.
metadata_size: ulong;
}
table PlasmaRefreshLRURequest {
// ID of the objects to be bumped in the LRU cache.
object_ids: [string];
}
table PlasmaRefreshLRUReply {
}

177
src/ray/plasma/plasma.h Normal file
View file

@ -0,0 +1,177 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_PLASMA_H
#define PLASMA_PLASMA_H
#include <errno.h>
#include <inttypes.h>
#include <stdbool.h>
#include <stddef.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <memory>
#include <string>
#include <unordered_map>
#include <unordered_set>
#include <vector>
#include "plasma/compat.h"
#include "arrow/status.h"
#include "arrow/util/logging.h"
#include "arrow/util/macros.h"
#include "plasma/common.h"
#ifdef PLASMA_CUDA
using arrow::cuda::CudaIpcMemHandle;
#endif
namespace plasma {
namespace flatbuf {
struct ObjectInfoT;
} // namespace flatbuf
#define HANDLE_SIGPIPE(s, fd_) \
do { \
Status _s = (s); \
if (!_s.ok()) { \
if (errno == EPIPE || errno == EBADF || errno == ECONNRESET) { \
ARROW_LOG(WARNING) \
<< "Received SIGPIPE, BAD FILE DESCRIPTOR, or ECONNRESET when " \
"sending a message to client on fd " \
<< fd_ \
<< ". " \
"The client on the other end may have hung up."; \
} else { \
return _s; \
} \
} \
} while (0);
/// Allocation granularity used in plasma for object allocation.
constexpr int64_t kBlockSize = 64;
/// Contains all information that is associated with a Plasma store client.
struct Client {
explicit Client(int fd);
/// The file descriptor used to communicate with the client.
int fd;
/// Object ids that are used by this client.
std::unordered_set<ObjectID> object_ids;
/// File descriptors that are used by this client.
std::unordered_set<int> used_fds;
/// The file descriptor used to push notifications to client. This is only valid
/// if client subscribes to plasma store. -1 indicates invalid.
int notification_fd;
std::string name = "anonymous_client";
};
// TODO(pcm): Replace this by the flatbuffers message PlasmaObjectSpec.
struct PlasmaObject {
#ifdef PLASMA_CUDA
// IPC handle for Cuda.
std::shared_ptr<CudaIpcMemHandle> ipc_handle;
#endif
/// The file descriptor of the memory mapped file in the store. It is used as
/// a unique identifier of the file in the client to look up the corresponding
/// file descriptor on the client's side.
int store_fd;
/// The offset in bytes in the memory mapped file of the data.
ptrdiff_t data_offset;
/// The offset in bytes in the memory mapped file of the metadata.
ptrdiff_t metadata_offset;
/// The size in bytes of the data.
int64_t data_size;
/// The size in bytes of the metadata.
int64_t metadata_size;
/// Device number object is on.
int device_num;
bool operator==(const PlasmaObject& other) const {
return (
#ifdef PLASMA_CUDA
(ipc_handle == other.ipc_handle) &&
#endif
(store_fd == other.store_fd) && (data_offset == other.data_offset) &&
(metadata_offset == other.metadata_offset) && (data_size == other.data_size) &&
(metadata_size == other.metadata_size) && (device_num == other.device_num));
}
};
enum class ObjectStatus : int {
/// The object was not found.
OBJECT_NOT_FOUND = 0,
/// The object was found.
OBJECT_FOUND = 1
};
/// The plasma store information that is exposed to the eviction policy.
struct PlasmaStoreInfo {
/// Objects that are in the Plasma store.
ObjectTable objects;
/// Boolean flag indicating whether to start the object store with hugepages
/// support enabled. Huge pages are substantially larger than normal memory
/// pages (e.g. 2MB or 1GB instead of 4KB) and using them can reduce
/// bookkeeping overhead from the OS.
bool hugepages_enabled;
/// A (platform-dependent) directory where to create the memory-backed file.
std::string directory;
};
/// Get an entry from the object table and return NULL if the object_id
/// is not present.
///
/// \param store_info The PlasmaStoreInfo that contains the object table.
/// \param object_id The object_id of the entry we are looking for.
/// \return The entry associated with the object_id or NULL if the object_id
/// is not present.
ObjectTableEntry* GetObjectTableEntry(PlasmaStoreInfo* store_info,
const ObjectID& object_id);
/// Print a warning if the status is less than zero. This should be used to check
/// the success of messages sent to plasma clients. We print a warning instead of
/// failing because the plasma clients are allowed to die. This is used to handle
/// situations where the store writes to a client file descriptor, and the client
/// may already have disconnected. If we have processed the disconnection and
/// closed the file descriptor, we should get a BAD FILE DESCRIPTOR error. If we
/// have not, then we should get a SIGPIPE. If we write to a TCP socket that
/// isn't connected yet, then we should get an ECONNRESET.
///
/// \param status The status to check. If it is less less than zero, we will
/// print a warning.
/// \param client_sock The client socket. This is just used to print some extra
/// information.
/// \return The errno set.
int WarnIfSigpipe(int status, int client_sock);
std::unique_ptr<uint8_t[]> CreateObjectInfoBuffer(flatbuf::ObjectInfoT* object_info);
std::unique_ptr<uint8_t[]> CreatePlasmaNotificationBuffer(
std::vector<flatbuf::ObjectInfoT>& object_info);
} // namespace plasma
#endif // PLASMA_PLASMA_H

View file

@ -0,0 +1,56 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include <arrow/util/logging.h>
#include "plasma/malloc.h"
#include "plasma/plasma_allocator.h"
namespace plasma {
extern "C" {
void* dlmemalign(size_t alignment, size_t bytes);
void dlfree(void* mem);
}
int64_t PlasmaAllocator::footprint_limit_ = 0;
int64_t PlasmaAllocator::allocated_ = 0;
void* PlasmaAllocator::Memalign(size_t alignment, size_t bytes) {
if (allocated_ + static_cast<int64_t>(bytes) > footprint_limit_) {
return nullptr;
}
void* mem = dlmemalign(alignment, bytes);
ARROW_CHECK(mem);
allocated_ += bytes;
return mem;
}
void PlasmaAllocator::Free(void* mem, size_t bytes) {
dlfree(mem);
allocated_ -= bytes;
}
void PlasmaAllocator::SetFootprintLimit(size_t bytes) {
footprint_limit_ = static_cast<int64_t>(bytes);
}
int64_t PlasmaAllocator::GetFootprintLimit() { return footprint_limit_; }
int64_t PlasmaAllocator::Allocated() { return allocated_; }
} // namespace plasma

View file

@ -0,0 +1,64 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_ALLOCATOR_H
#define PLASMA_ALLOCATOR_H
#include <cstddef>
#include <cstdint>
namespace plasma {
class PlasmaAllocator {
public:
/// Allocates size bytes and returns a pointer to the allocated memory. The
/// memory address will be a multiple of alignment, which must be a power of two.
///
/// \param alignment Memory alignment.
/// \param bytes Number of bytes.
/// \return Pointer to allocated memory.
static void* Memalign(size_t alignment, size_t bytes);
/// Frees the memory space pointed to by mem, which must have been returned by
/// a previous call to Memalign()
///
/// \param mem Pointer to memory to free.
/// \param bytes Number of bytes to be freed.
static void Free(void* mem, size_t bytes);
/// Sets the memory footprint limit for Plasma.
///
/// \param bytes Plasma memory footprint limit in bytes.
static void SetFootprintLimit(size_t bytes);
/// Get the memory footprint limit for Plasma.
///
/// \return Plasma memory footprint limit in bytes.
static int64_t GetFootprintLimit();
/// Get the number of bytes allocated by Plasma so far.
/// \return Number of bytes allocated by Plasma so far.
static int64_t Allocated();
private:
static int64_t allocated_;
static int64_t footprint_limit_;
};
} // namespace plasma
#endif // ARROW_PLASMA_ALLOCATOR_H

831
src/ray/plasma/protocol.cc Normal file
View file

@ -0,0 +1,831 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/protocol.h"
#include <utility>
#include "flatbuffers/flatbuffers.h"
#include "plasma/common.h"
#include "plasma/io.h"
#include "plasma/plasma_generated.h"
#ifdef PLASMA_CUDA
#include "arrow/gpu/cuda_api.h"
#endif
#include "arrow/util/ubsan.h"
namespace fb = plasma::flatbuf;
namespace plasma {
using fb::MessageType;
using fb::PlasmaError;
using fb::PlasmaObjectSpec;
using flatbuffers::uoffset_t;
#define PLASMA_CHECK_ENUM(x, y) \
static_assert(static_cast<int>(x) == static_cast<int>(y), "protocol mismatch")
flatbuffers::Offset<flatbuffers::Vector<flatbuffers::Offset<flatbuffers::String>>>
ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb, const ObjectID* object_ids,
int64_t num_objects) {
std::vector<flatbuffers::Offset<flatbuffers::String>> results;
for (int64_t i = 0; i < num_objects; i++) {
results.push_back(fbb->CreateString(object_ids[i].binary()));
}
return fbb->CreateVector(arrow::util::MakeNonNull(results.data()), results.size());
}
flatbuffers::Offset<flatbuffers::Vector<flatbuffers::Offset<flatbuffers::String>>>
ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb,
const std::vector<std::string>& strings) {
std::vector<flatbuffers::Offset<flatbuffers::String>> results;
for (size_t i = 0; i < strings.size(); i++) {
results.push_back(fbb->CreateString(strings[i]));
}
return fbb->CreateVector(arrow::util::MakeNonNull(results.data()), results.size());
}
flatbuffers::Offset<flatbuffers::Vector<int64_t>> ToFlatbuffer(
flatbuffers::FlatBufferBuilder* fbb, const std::vector<int64_t>& data) {
return fbb->CreateVector(arrow::util::MakeNonNull(data.data()), data.size());
}
Status PlasmaReceive(int sock, MessageType message_type, std::vector<uint8_t>* buffer) {
MessageType type;
RETURN_NOT_OK(ReadMessage(sock, &type, buffer));
ARROW_CHECK(type == message_type)
<< "type = " << static_cast<int64_t>(type)
<< ", message_type = " << static_cast<int64_t>(message_type);
return Status::OK();
}
// Helper function to create a vector of elements from Data (Request/Reply struct).
// The Getter function is used to extract one element from Data.
template <typename T, typename Data, typename Getter>
void ToVector(const Data& request, std::vector<T>* out, const Getter& getter) {
int count = request.count();
out->clear();
out->reserve(count);
for (int i = 0; i < count; ++i) {
out->push_back(getter(request, i));
}
}
template <typename T, typename FlatbufferVectorPointer, typename Converter>
void ConvertToVector(const FlatbufferVectorPointer fbvector, std::vector<T>* out,
const Converter& converter) {
out->clear();
out->reserve(fbvector->size());
for (size_t i = 0; i < fbvector->size(); ++i) {
out->push_back(converter(*fbvector->Get(i)));
}
}
template <typename Message>
Status PlasmaSend(int sock, MessageType message_type, flatbuffers::FlatBufferBuilder* fbb,
const Message& message) {
fbb->Finish(message);
return WriteMessage(sock, message_type, fbb->GetSize(), fbb->GetBufferPointer());
}
Status PlasmaErrorStatus(fb::PlasmaError plasma_error) {
switch (plasma_error) {
case fb::PlasmaError::OK:
return Status::OK();
case fb::PlasmaError::ObjectExists:
return MakePlasmaError(PlasmaErrorCode::PlasmaObjectExists,
"object already exists in the plasma store");
case fb::PlasmaError::ObjectNonexistent:
return MakePlasmaError(PlasmaErrorCode::PlasmaObjectNonexistent,
"object does not exist in the plasma store");
case fb::PlasmaError::OutOfMemory:
return MakePlasmaError(PlasmaErrorCode::PlasmaStoreFull,
"object does not fit in the plasma store");
default:
ARROW_LOG(FATAL) << "unknown plasma error code " << static_cast<int>(plasma_error);
}
return Status::OK();
}
// Set options messages.
Status SendSetOptionsRequest(int sock, const std::string& client_name,
int64_t output_memory_limit) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaSetOptionsRequest(fbb, fbb.CreateString(client_name),
output_memory_limit);
return PlasmaSend(sock, MessageType::PlasmaSetOptionsRequest, &fbb, message);
}
Status ReadSetOptionsRequest(uint8_t* data, size_t size, std::string* client_name,
int64_t* output_memory_quota) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaSetOptionsRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*client_name = std::string(message->client_name()->str());
*output_memory_quota = message->output_memory_quota();
return Status::OK();
}
Status SendSetOptionsReply(int sock, PlasmaError error) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaSetOptionsReply(fbb, error);
return PlasmaSend(sock, MessageType::PlasmaSetOptionsReply, &fbb, message);
}
Status ReadSetOptionsReply(uint8_t* data, size_t size) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaSetOptionsReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
return PlasmaErrorStatus(message->error());
}
// Get debug string messages.
Status SendGetDebugStringRequest(int sock) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaGetDebugStringRequest(fbb);
return PlasmaSend(sock, MessageType::PlasmaGetDebugStringRequest, &fbb, message);
}
Status SendGetDebugStringReply(int sock, const std::string& debug_string) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaGetDebugStringReply(fbb, fbb.CreateString(debug_string));
return PlasmaSend(sock, MessageType::PlasmaGetDebugStringReply, &fbb, message);
}
Status ReadGetDebugStringReply(uint8_t* data, size_t size, std::string* debug_string) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaGetDebugStringReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*debug_string = message->debug_string()->str();
return Status::OK();
}
// Create messages.
Status SendCreateRequest(int sock, ObjectID object_id, bool evict_if_full,
int64_t data_size, int64_t metadata_size, int device_num) {
flatbuffers::FlatBufferBuilder fbb;
auto message =
fb::CreatePlasmaCreateRequest(fbb, fbb.CreateString(object_id.binary()),
evict_if_full, data_size, metadata_size, device_num);
return PlasmaSend(sock, MessageType::PlasmaCreateRequest, &fbb, message);
}
Status ReadCreateRequest(uint8_t* data, size_t size, ObjectID* object_id,
bool* evict_if_full, int64_t* data_size, int64_t* metadata_size,
int* device_num) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaCreateRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*evict_if_full = message->evict_if_full();
*data_size = message->data_size();
*metadata_size = message->metadata_size();
*object_id = ObjectID::from_binary(message->object_id()->str());
*device_num = message->device_num();
return Status::OK();
}
Status SendCreateReply(int sock, ObjectID object_id, PlasmaObject* object,
PlasmaError error_code, int64_t mmap_size) {
flatbuffers::FlatBufferBuilder fbb;
PlasmaObjectSpec plasma_object(object->store_fd, object->data_offset, object->data_size,
object->metadata_offset, object->metadata_size,
object->device_num);
auto object_string = fbb.CreateString(object_id.binary());
#ifdef PLASMA_CUDA
flatbuffers::Offset<fb::CudaHandle> ipc_handle;
if (object->device_num != 0) {
std::shared_ptr<arrow::Buffer> handle;
ARROW_ASSIGN_OR_RAISE(handle, object->ipc_handle->Serialize());
ipc_handle =
fb::CreateCudaHandle(fbb, fbb.CreateVector(handle->data(), handle->size()));
}
#endif
fb::PlasmaCreateReplyBuilder crb(fbb);
crb.add_error(static_cast<PlasmaError>(error_code));
crb.add_plasma_object(&plasma_object);
crb.add_object_id(object_string);
crb.add_store_fd(object->store_fd);
crb.add_mmap_size(mmap_size);
if (object->device_num != 0) {
#ifdef PLASMA_CUDA
crb.add_ipc_handle(ipc_handle);
#else
ARROW_LOG(FATAL) << "This should be unreachable.";
#endif
}
auto message = crb.Finish();
return PlasmaSend(sock, MessageType::PlasmaCreateReply, &fbb, message);
}
Status ReadCreateReply(uint8_t* data, size_t size, ObjectID* object_id,
PlasmaObject* object, int* store_fd, int64_t* mmap_size) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaCreateReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
object->store_fd = message->plasma_object()->segment_index();
object->data_offset = message->plasma_object()->data_offset();
object->data_size = message->plasma_object()->data_size();
object->metadata_offset = message->plasma_object()->metadata_offset();
object->metadata_size = message->plasma_object()->metadata_size();
*store_fd = message->store_fd();
*mmap_size = message->mmap_size();
object->device_num = message->plasma_object()->device_num();
#ifdef PLASMA_CUDA
if (object->device_num != 0) {
ARROW_ASSIGN_OR_RAISE(
object->ipc_handle,
CudaIpcMemHandle::FromBuffer(message->ipc_handle()->handle()->data()));
}
#endif
return PlasmaErrorStatus(message->error());
}
Status SendCreateAndSealRequest(int sock, const ObjectID& object_id, bool evict_if_full,
const std::string& data, const std::string& metadata,
unsigned char* digest) {
flatbuffers::FlatBufferBuilder fbb;
auto digest_string = fbb.CreateString(reinterpret_cast<char*>(digest), kDigestSize);
auto message = fb::CreatePlasmaCreateAndSealRequest(
fbb, fbb.CreateString(object_id.binary()), evict_if_full, fbb.CreateString(data),
fbb.CreateString(metadata), digest_string);
return PlasmaSend(sock, MessageType::PlasmaCreateAndSealRequest, &fbb, message);
}
Status ReadCreateAndSealRequest(uint8_t* data, size_t size, ObjectID* object_id,
bool* evict_if_full, std::string* object_data,
std::string* metadata, std::string* digest) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaCreateAndSealRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
*evict_if_full = message->evict_if_full();
*object_data = message->data()->str();
*metadata = message->metadata()->str();
ARROW_CHECK(message->digest()->size() == kDigestSize);
digest->assign(message->digest()->data(), kDigestSize);
return Status::OK();
}
Status SendCreateAndSealBatchRequest(int sock, const std::vector<ObjectID>& object_ids,
bool evict_if_full,
const std::vector<std::string>& data,
const std::vector<std::string>& metadata,
const std::vector<std::string>& digests) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaCreateAndSealBatchRequest(
fbb, ToFlatbuffer(&fbb, object_ids.data(), object_ids.size()), evict_if_full,
ToFlatbuffer(&fbb, data), ToFlatbuffer(&fbb, metadata),
ToFlatbuffer(&fbb, digests));
return PlasmaSend(sock, MessageType::PlasmaCreateAndSealBatchRequest, &fbb, message);
}
Status ReadCreateAndSealBatchRequest(uint8_t* data, size_t size,
std::vector<ObjectID>* object_ids,
bool* evict_if_full,
std::vector<std::string>* object_data,
std::vector<std::string>* metadata,
std::vector<std::string>* digests) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaCreateAndSealBatchRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*evict_if_full = message->evict_if_full();
ConvertToVector(message->object_ids(), object_ids,
[](const flatbuffers::String& element) {
return ObjectID::from_binary(element.str());
});
ConvertToVector(message->data(), object_data,
[](const flatbuffers::String& element) { return element.str(); });
ConvertToVector(message->metadata(), metadata,
[](const flatbuffers::String& element) { return element.str(); });
ConvertToVector(message->digest(), digests,
[](const flatbuffers::String& element) { return element.str(); });
return Status::OK();
}
Status SendCreateAndSealReply(int sock, PlasmaError error) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaCreateAndSealReply(fbb, static_cast<PlasmaError>(error));
return PlasmaSend(sock, MessageType::PlasmaCreateAndSealReply, &fbb, message);
}
Status ReadCreateAndSealReply(uint8_t* data, size_t size) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaCreateAndSealReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
return PlasmaErrorStatus(message->error());
}
Status SendCreateAndSealBatchReply(int sock, PlasmaError error) {
flatbuffers::FlatBufferBuilder fbb;
auto message =
fb::CreatePlasmaCreateAndSealBatchReply(fbb, static_cast<PlasmaError>(error));
return PlasmaSend(sock, MessageType::PlasmaCreateAndSealBatchReply, &fbb, message);
}
Status ReadCreateAndSealBatchReply(uint8_t* data, size_t size) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaCreateAndSealBatchReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
return PlasmaErrorStatus(message->error());
}
Status SendAbortRequest(int sock, ObjectID object_id) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaAbortRequest(fbb, fbb.CreateString(object_id.binary()));
return PlasmaSend(sock, MessageType::PlasmaAbortRequest, &fbb, message);
}
Status ReadAbortRequest(uint8_t* data, size_t size, ObjectID* object_id) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaAbortRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
return Status::OK();
}
Status SendAbortReply(int sock, ObjectID object_id) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaAbortReply(fbb, fbb.CreateString(object_id.binary()));
return PlasmaSend(sock, MessageType::PlasmaAbortReply, &fbb, message);
}
Status ReadAbortReply(uint8_t* data, size_t size, ObjectID* object_id) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaAbortReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
return Status::OK();
}
// Seal messages.
Status SendSealRequest(int sock, ObjectID object_id, const std::string& digest) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaSealRequest(fbb, fbb.CreateString(object_id.binary()),
fbb.CreateString(digest));
return PlasmaSend(sock, MessageType::PlasmaSealRequest, &fbb, message);
}
Status ReadSealRequest(uint8_t* data, size_t size, ObjectID* object_id,
std::string* digest) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaSealRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
ARROW_CHECK_EQ(message->digest()->size(), kDigestSize);
digest->assign(message->digest()->data(), kDigestSize);
return Status::OK();
}
Status SendSealReply(int sock, ObjectID object_id, PlasmaError error) {
flatbuffers::FlatBufferBuilder fbb;
auto message =
fb::CreatePlasmaSealReply(fbb, fbb.CreateString(object_id.binary()), error);
return PlasmaSend(sock, MessageType::PlasmaSealReply, &fbb, message);
}
Status ReadSealReply(uint8_t* data, size_t size, ObjectID* object_id) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaSealReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
return PlasmaErrorStatus(message->error());
}
// Release messages.
Status SendReleaseRequest(int sock, ObjectID object_id) {
flatbuffers::FlatBufferBuilder fbb;
auto message =
fb::CreatePlasmaReleaseRequest(fbb, fbb.CreateString(object_id.binary()));
return PlasmaSend(sock, MessageType::PlasmaReleaseRequest, &fbb, message);
}
Status ReadReleaseRequest(uint8_t* data, size_t size, ObjectID* object_id) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaReleaseRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
return Status::OK();
}
Status SendReleaseReply(int sock, ObjectID object_id, PlasmaError error) {
flatbuffers::FlatBufferBuilder fbb;
auto message =
fb::CreatePlasmaReleaseReply(fbb, fbb.CreateString(object_id.binary()), error);
return PlasmaSend(sock, MessageType::PlasmaReleaseReply, &fbb, message);
}
Status ReadReleaseReply(uint8_t* data, size_t size, ObjectID* object_id) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaReleaseReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
return PlasmaErrorStatus(message->error());
}
// Delete objects messages.
Status SendDeleteRequest(int sock, const std::vector<ObjectID>& object_ids) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaDeleteRequest(
fbb, static_cast<int32_t>(object_ids.size()),
ToFlatbuffer(&fbb, &object_ids[0], object_ids.size()));
return PlasmaSend(sock, MessageType::PlasmaDeleteRequest, &fbb, message);
}
Status ReadDeleteRequest(uint8_t* data, size_t size, std::vector<ObjectID>* object_ids) {
using fb::PlasmaDeleteRequest;
DCHECK(data);
DCHECK(object_ids);
auto message = flatbuffers::GetRoot<PlasmaDeleteRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
ToVector(*message, object_ids, [](const PlasmaDeleteRequest& request, int i) {
return ObjectID::from_binary(request.object_ids()->Get(i)->str());
});
return Status::OK();
}
Status SendDeleteReply(int sock, const std::vector<ObjectID>& object_ids,
const std::vector<PlasmaError>& errors) {
DCHECK(object_ids.size() == errors.size());
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaDeleteReply(
fbb, static_cast<int32_t>(object_ids.size()),
ToFlatbuffer(&fbb, &object_ids[0], object_ids.size()),
fbb.CreateVector(
arrow::util::MakeNonNull(reinterpret_cast<const int32_t*>(errors.data())),
object_ids.size()));
return PlasmaSend(sock, MessageType::PlasmaDeleteReply, &fbb, message);
}
Status ReadDeleteReply(uint8_t* data, size_t size, std::vector<ObjectID>* object_ids,
std::vector<PlasmaError>* errors) {
using fb::PlasmaDeleteReply;
DCHECK(data);
DCHECK(object_ids);
DCHECK(errors);
auto message = flatbuffers::GetRoot<PlasmaDeleteReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
ToVector(*message, object_ids, [](const PlasmaDeleteReply& request, int i) {
return ObjectID::from_binary(request.object_ids()->Get(i)->str());
});
ToVector(*message, errors, [](const PlasmaDeleteReply& request, int i) {
return static_cast<PlasmaError>(request.errors()->data()[i]);
});
return Status::OK();
}
// Contains messages.
Status SendContainsRequest(int sock, ObjectID object_id) {
flatbuffers::FlatBufferBuilder fbb;
auto message =
fb::CreatePlasmaContainsRequest(fbb, fbb.CreateString(object_id.binary()));
return PlasmaSend(sock, MessageType::PlasmaContainsRequest, &fbb, message);
}
Status ReadContainsRequest(uint8_t* data, size_t size, ObjectID* object_id) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaContainsRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
return Status::OK();
}
Status SendContainsReply(int sock, ObjectID object_id, bool has_object) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaContainsReply(fbb, fbb.CreateString(object_id.binary()),
has_object);
return PlasmaSend(sock, MessageType::PlasmaContainsReply, &fbb, message);
}
Status ReadContainsReply(uint8_t* data, size_t size, ObjectID* object_id,
bool* has_object) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaContainsReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
*has_object = message->has_object();
return Status::OK();
}
// List messages.
Status SendListRequest(int sock) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaListRequest(fbb);
return PlasmaSend(sock, MessageType::PlasmaListRequest, &fbb, message);
}
Status ReadListRequest(uint8_t* data, size_t size) { return Status::OK(); }
Status SendListReply(int sock, const ObjectTable& objects) {
flatbuffers::FlatBufferBuilder fbb;
std::vector<flatbuffers::Offset<fb::ObjectInfo>> object_infos;
for (auto const& entry : objects) {
auto digest = entry.second->state == ObjectState::PLASMA_CREATED
? fbb.CreateString("")
: fbb.CreateString(reinterpret_cast<char*>(entry.second->digest),
kDigestSize);
auto info = fb::CreateObjectInfo(fbb, fbb.CreateString(entry.first.binary()),
entry.second->data_size, entry.second->metadata_size,
entry.second->ref_count, entry.second->create_time,
entry.second->construct_duration, digest);
object_infos.push_back(info);
}
auto message = fb::CreatePlasmaListReply(
fbb, fbb.CreateVector(arrow::util::MakeNonNull(object_infos.data()),
object_infos.size()));
return PlasmaSend(sock, MessageType::PlasmaListReply, &fbb, message);
}
Status ReadListReply(uint8_t* data, size_t size, ObjectTable* objects) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaListReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
for (auto const& object : *message->objects()) {
ObjectID object_id = ObjectID::from_binary(object->object_id()->str());
auto entry = std::unique_ptr<ObjectTableEntry>(new ObjectTableEntry());
entry->data_size = object->data_size();
entry->metadata_size = object->metadata_size();
entry->ref_count = object->ref_count();
entry->create_time = object->create_time();
entry->construct_duration = object->construct_duration();
entry->state = object->digest()->size() == 0 ? ObjectState::PLASMA_CREATED
: ObjectState::PLASMA_SEALED;
(*objects)[object_id] = std::move(entry);
}
return Status::OK();
}
// Connect messages.
Status SendConnectRequest(int sock) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaConnectRequest(fbb);
return PlasmaSend(sock, MessageType::PlasmaConnectRequest, &fbb, message);
}
Status ReadConnectRequest(uint8_t* data) { return Status::OK(); }
Status SendConnectReply(int sock, int64_t memory_capacity) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaConnectReply(fbb, memory_capacity);
return PlasmaSend(sock, MessageType::PlasmaConnectReply, &fbb, message);
}
Status ReadConnectReply(uint8_t* data, size_t size, int64_t* memory_capacity) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaConnectReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*memory_capacity = message->memory_capacity();
return Status::OK();
}
// Evict messages.
Status SendEvictRequest(int sock, int64_t num_bytes) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaEvictRequest(fbb, num_bytes);
return PlasmaSend(sock, MessageType::PlasmaEvictRequest, &fbb, message);
}
Status ReadEvictRequest(uint8_t* data, size_t size, int64_t* num_bytes) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaEvictRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*num_bytes = message->num_bytes();
return Status::OK();
}
Status SendEvictReply(int sock, int64_t num_bytes) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaEvictReply(fbb, num_bytes);
return PlasmaSend(sock, MessageType::PlasmaEvictReply, &fbb, message);
}
Status ReadEvictReply(uint8_t* data, size_t size, int64_t& num_bytes) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaEvictReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
num_bytes = message->num_bytes();
return Status::OK();
}
// Get messages.
Status SendGetRequest(int sock, const ObjectID* object_ids, int64_t num_objects,
int64_t timeout_ms) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaGetRequest(
fbb, ToFlatbuffer(&fbb, object_ids, num_objects), timeout_ms);
return PlasmaSend(sock, MessageType::PlasmaGetRequest, &fbb, message);
}
Status ReadGetRequest(uint8_t* data, size_t size, std::vector<ObjectID>& object_ids,
int64_t* timeout_ms) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaGetRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
for (uoffset_t i = 0; i < message->object_ids()->size(); ++i) {
auto object_id = message->object_ids()->Get(i)->str();
object_ids.push_back(ObjectID::from_binary(object_id));
}
*timeout_ms = message->timeout_ms();
return Status::OK();
}
Status SendGetReply(int sock, ObjectID object_ids[],
std::unordered_map<ObjectID, PlasmaObject>& plasma_objects,
int64_t num_objects, const std::vector<int>& store_fds,
const std::vector<int64_t>& mmap_sizes) {
flatbuffers::FlatBufferBuilder fbb;
std::vector<PlasmaObjectSpec> objects;
std::vector<flatbuffers::Offset<fb::CudaHandle>> handles;
for (int64_t i = 0; i < num_objects; ++i) {
const PlasmaObject& object = plasma_objects[object_ids[i]];
objects.push_back(PlasmaObjectSpec(object.store_fd, object.data_offset,
object.data_size, object.metadata_offset,
object.metadata_size, object.device_num));
#ifdef PLASMA_CUDA
if (object.device_num != 0) {
std::shared_ptr<arrow::Buffer> handle;
ARROW_ASSIGN_OR_RAISE(handle, object.ipc_handle->Serialize());
handles.push_back(
fb::CreateCudaHandle(fbb, fbb.CreateVector(handle->data(), handle->size())));
}
#endif
}
auto message = fb::CreatePlasmaGetReply(
fbb, ToFlatbuffer(&fbb, object_ids, num_objects),
fbb.CreateVectorOfStructs(arrow::util::MakeNonNull(objects.data()), num_objects),
fbb.CreateVector(arrow::util::MakeNonNull(store_fds.data()), store_fds.size()),
fbb.CreateVector(arrow::util::MakeNonNull(mmap_sizes.data()), mmap_sizes.size()),
fbb.CreateVector(arrow::util::MakeNonNull(handles.data()), handles.size()));
return PlasmaSend(sock, MessageType::PlasmaGetReply, &fbb, message);
}
Status ReadGetReply(uint8_t* data, size_t size, ObjectID object_ids[],
PlasmaObject plasma_objects[], int64_t num_objects,
std::vector<int>& store_fds, std::vector<int64_t>& mmap_sizes) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaGetReply>(data);
#ifdef PLASMA_CUDA
int handle_pos = 0;
#endif
DCHECK(VerifyFlatbuffer(message, data, size));
for (uoffset_t i = 0; i < num_objects; ++i) {
object_ids[i] = ObjectID::from_binary(message->object_ids()->Get(i)->str());
}
for (uoffset_t i = 0; i < num_objects; ++i) {
const PlasmaObjectSpec* object = message->plasma_objects()->Get(i);
plasma_objects[i].store_fd = object->segment_index();
plasma_objects[i].data_offset = object->data_offset();
plasma_objects[i].data_size = object->data_size();
plasma_objects[i].metadata_offset = object->metadata_offset();
plasma_objects[i].metadata_size = object->metadata_size();
plasma_objects[i].device_num = object->device_num();
#ifdef PLASMA_CUDA
if (object->device_num() != 0) {
const void* ipc_handle = message->handles()->Get(handle_pos)->handle()->data();
ARROW_ASSIGN_OR_RAISE(plasma_objects[i].ipc_handle,
CudaIpcMemHandle::FromBuffer(ipc_handle));
handle_pos++;
}
#endif
}
ARROW_CHECK(message->store_fds()->size() == message->mmap_sizes()->size());
for (uoffset_t i = 0; i < message->store_fds()->size(); i++) {
store_fds.push_back(message->store_fds()->Get(i));
mmap_sizes.push_back(message->mmap_sizes()->Get(i));
}
return Status::OK();
}
// Subscribe messages.
Status SendSubscribeRequest(int sock) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaSubscribeRequest(fbb);
return PlasmaSend(sock, MessageType::PlasmaSubscribeRequest, &fbb, message);
}
// Data messages.
Status SendDataRequest(int sock, ObjectID object_id, const char* address, int port) {
flatbuffers::FlatBufferBuilder fbb;
auto addr = fbb.CreateString(address, strlen(address));
auto message =
fb::CreatePlasmaDataRequest(fbb, fbb.CreateString(object_id.binary()), addr, port);
return PlasmaSend(sock, MessageType::PlasmaDataRequest, &fbb, message);
}
Status ReadDataRequest(uint8_t* data, size_t size, ObjectID* object_id, char** address,
int* port) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaDataRequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
DCHECK(message->object_id()->size() == sizeof(ObjectID));
*object_id = ObjectID::from_binary(message->object_id()->str());
#ifdef _WIN32
*address = _strdup(message->address()->c_str());
#else
*address = strdup(message->address()->c_str());
#endif
*port = message->port();
return Status::OK();
}
Status SendDataReply(int sock, ObjectID object_id, int64_t object_size,
int64_t metadata_size) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaDataReply(fbb, fbb.CreateString(object_id.binary()),
object_size, metadata_size);
return PlasmaSend(sock, MessageType::PlasmaDataReply, &fbb, message);
}
Status ReadDataReply(uint8_t* data, size_t size, ObjectID* object_id,
int64_t* object_size, int64_t* metadata_size) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaDataReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
*object_id = ObjectID::from_binary(message->object_id()->str());
*object_size = static_cast<int64_t>(message->object_size());
*metadata_size = static_cast<int64_t>(message->metadata_size());
return Status::OK();
}
// RefreshLRU messages.
Status SendRefreshLRURequest(int sock, const std::vector<ObjectID>& object_ids) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaRefreshLRURequest(
fbb, ToFlatbuffer(&fbb, object_ids.data(), object_ids.size()));
return PlasmaSend(sock, MessageType::PlasmaRefreshLRURequest, &fbb, message);
}
Status ReadRefreshLRURequest(uint8_t* data, size_t size,
std::vector<ObjectID>* object_ids) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaRefreshLRURequest>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
for (uoffset_t i = 0; i < message->object_ids()->size(); ++i) {
auto object_id = message->object_ids()->Get(i)->str();
object_ids->push_back(ObjectID::from_binary(object_id));
}
return Status::OK();
}
Status SendRefreshLRUReply(int sock) {
flatbuffers::FlatBufferBuilder fbb;
auto message = fb::CreatePlasmaRefreshLRUReply(fbb);
return PlasmaSend(sock, MessageType::PlasmaRefreshLRUReply, &fbb, message);
}
Status ReadRefreshLRUReply(uint8_t* data, size_t size) {
DCHECK(data);
auto message = flatbuffers::GetRoot<fb::PlasmaRefreshLRUReply>(data);
DCHECK(VerifyFlatbuffer(message, data, size));
return Status::OK();
}
} // namespace plasma

251
src/ray/plasma/protocol.h Normal file
View file

@ -0,0 +1,251 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_PROTOCOL_H
#define PLASMA_PROTOCOL_H
#include <memory>
#include <string>
#include <unordered_map>
#include <vector>
#include "arrow/status.h"
#include "plasma/plasma.h"
#include "plasma/plasma_generated.h"
namespace plasma {
using arrow::Status;
using flatbuf::MessageType;
using flatbuf::PlasmaError;
template <class T>
bool VerifyFlatbuffer(T* object, uint8_t* data, size_t size) {
flatbuffers::Verifier verifier(data, size);
return object->Verify(verifier);
}
flatbuffers::Offset<flatbuffers::Vector<flatbuffers::Offset<flatbuffers::String>>>
ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb, const ObjectID* object_ids,
int64_t num_objects);
flatbuffers::Offset<flatbuffers::Vector<flatbuffers::Offset<flatbuffers::String>>>
ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb,
const std::vector<std::string>& strings);
flatbuffers::Offset<flatbuffers::Vector<int64_t>> ToFlatbuffer(
flatbuffers::FlatBufferBuilder* fbb, const std::vector<int64_t>& data);
/* Plasma receive message. */
Status PlasmaReceive(int sock, MessageType message_type, std::vector<uint8_t>* buffer);
/* Set options messages. */
Status SendSetOptionsRequest(int sock, const std::string& client_name,
int64_t output_memory_limit);
Status ReadSetOptionsRequest(uint8_t* data, size_t size, std::string* client_name,
int64_t* output_memory_quota);
Status SendSetOptionsReply(int sock, PlasmaError error);
Status ReadSetOptionsReply(uint8_t* data, size_t size);
/* Debug string messages. */
Status SendGetDebugStringRequest(int sock);
Status SendGetDebugStringReply(int sock, const std::string& debug_string);
Status ReadGetDebugStringReply(uint8_t* data, size_t size, std::string* debug_string);
/* Plasma Create message functions. */
Status SendCreateRequest(int sock, ObjectID object_id, bool evict_if_full,
int64_t data_size, int64_t metadata_size, int device_num);
Status ReadCreateRequest(uint8_t* data, size_t size, ObjectID* object_id,
bool* evict_if_full, int64_t* data_size, int64_t* metadata_size,
int* device_num);
Status SendCreateReply(int sock, ObjectID object_id, PlasmaObject* object,
PlasmaError error, int64_t mmap_size);
Status ReadCreateReply(uint8_t* data, size_t size, ObjectID* object_id,
PlasmaObject* object, int* store_fd, int64_t* mmap_size);
Status SendCreateAndSealRequest(int sock, const ObjectID& object_id, bool evict_if_full,
const std::string& data, const std::string& metadata,
unsigned char* digest);
Status ReadCreateAndSealRequest(uint8_t* data, size_t size, ObjectID* object_id,
bool* evict_if_full, std::string* object_data,
std::string* metadata, std::string* digest);
Status SendCreateAndSealBatchRequest(int sock, const std::vector<ObjectID>& object_ids,
bool evict_if_full,
const std::vector<std::string>& data,
const std::vector<std::string>& metadata,
const std::vector<std::string>& digests);
Status ReadCreateAndSealBatchRequest(uint8_t* data, size_t size,
std::vector<ObjectID>* object_id,
bool* evict_if_full,
std::vector<std::string>* object_data,
std::vector<std::string>* metadata,
std::vector<std::string>* digests);
Status SendCreateAndSealReply(int sock, PlasmaError error);
Status ReadCreateAndSealReply(uint8_t* data, size_t size);
Status SendCreateAndSealBatchReply(int sock, PlasmaError error);
Status ReadCreateAndSealBatchReply(uint8_t* data, size_t size);
Status SendAbortRequest(int sock, ObjectID object_id);
Status ReadAbortRequest(uint8_t* data, size_t size, ObjectID* object_id);
Status SendAbortReply(int sock, ObjectID object_id);
Status ReadAbortReply(uint8_t* data, size_t size, ObjectID* object_id);
/* Plasma Seal message functions. */
Status SendSealRequest(int sock, ObjectID object_id, const std::string& digest);
Status ReadSealRequest(uint8_t* data, size_t size, ObjectID* object_id,
std::string* digest);
Status SendSealReply(int sock, ObjectID object_id, PlasmaError error);
Status ReadSealReply(uint8_t* data, size_t size, ObjectID* object_id);
/* Plasma Get message functions. */
Status SendGetRequest(int sock, const ObjectID* object_ids, int64_t num_objects,
int64_t timeout_ms);
Status ReadGetRequest(uint8_t* data, size_t size, std::vector<ObjectID>& object_ids,
int64_t* timeout_ms);
Status SendGetReply(int sock, ObjectID object_ids[],
std::unordered_map<ObjectID, PlasmaObject>& plasma_objects,
int64_t num_objects, const std::vector<int>& store_fds,
const std::vector<int64_t>& mmap_sizes);
Status ReadGetReply(uint8_t* data, size_t size, ObjectID object_ids[],
PlasmaObject plasma_objects[], int64_t num_objects,
std::vector<int>& store_fds, std::vector<int64_t>& mmap_sizes);
/* Plasma Release message functions. */
Status SendReleaseRequest(int sock, ObjectID object_id);
Status ReadReleaseRequest(uint8_t* data, size_t size, ObjectID* object_id);
Status SendReleaseReply(int sock, ObjectID object_id, PlasmaError error);
Status ReadReleaseReply(uint8_t* data, size_t size, ObjectID* object_id);
/* Plasma Delete objects message functions. */
Status SendDeleteRequest(int sock, const std::vector<ObjectID>& object_ids);
Status ReadDeleteRequest(uint8_t* data, size_t size, std::vector<ObjectID>* object_ids);
Status SendDeleteReply(int sock, const std::vector<ObjectID>& object_ids,
const std::vector<PlasmaError>& errors);
Status ReadDeleteReply(uint8_t* data, size_t size, std::vector<ObjectID>* object_ids,
std::vector<PlasmaError>* errors);
/* Plasma Contains message functions. */
Status SendContainsRequest(int sock, ObjectID object_id);
Status ReadContainsRequest(uint8_t* data, size_t size, ObjectID* object_id);
Status SendContainsReply(int sock, ObjectID object_id, bool has_object);
Status ReadContainsReply(uint8_t* data, size_t size, ObjectID* object_id,
bool* has_object);
/* Plasma List message functions. */
Status SendListRequest(int sock);
Status ReadListRequest(uint8_t* data, size_t size);
Status SendListReply(int sock, const ObjectTable& objects);
Status ReadListReply(uint8_t* data, size_t size, ObjectTable* objects);
/* Plasma Connect message functions. */
Status SendConnectRequest(int sock);
Status ReadConnectRequest(uint8_t* data, size_t size);
Status SendConnectReply(int sock, int64_t memory_capacity);
Status ReadConnectReply(uint8_t* data, size_t size, int64_t* memory_capacity);
/* Plasma Evict message functions (no reply so far). */
Status SendEvictRequest(int sock, int64_t num_bytes);
Status ReadEvictRequest(uint8_t* data, size_t size, int64_t* num_bytes);
Status SendEvictReply(int sock, int64_t num_bytes);
Status ReadEvictReply(uint8_t* data, size_t size, int64_t& num_bytes);
/* Plasma Subscribe message functions. */
Status SendSubscribeRequest(int sock);
/* Data messages. */
Status SendDataRequest(int sock, ObjectID object_id, const char* address, int port);
Status ReadDataRequest(uint8_t* data, size_t size, ObjectID* object_id, char** address,
int* port);
Status SendDataReply(int sock, ObjectID object_id, int64_t object_size,
int64_t metadata_size);
Status ReadDataReply(uint8_t* data, size_t size, ObjectID* object_id,
int64_t* object_size, int64_t* metadata_size);
/* Plasma refresh LRU cache functions. */
Status SendRefreshLRURequest(int sock, const std::vector<ObjectID>& object_ids);
Status ReadRefreshLRURequest(uint8_t* data, size_t size,
std::vector<ObjectID>* object_ids);
Status SendRefreshLRUReply(int sock);
Status ReadRefreshLRUReply(uint8_t* data, size_t size);
} // namespace plasma
#endif /* PLASMA_PROTOCOL */

View file

@ -0,0 +1,177 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "plasma/quota_aware_policy.h"
#include "plasma/common.h"
#include "plasma/plasma_allocator.h"
#include <algorithm>
#include <memory>
#include <sstream>
namespace plasma {
QuotaAwarePolicy::QuotaAwarePolicy(PlasmaStoreInfo* store_info, int64_t max_size)
: EvictionPolicy(store_info, max_size) {}
bool QuotaAwarePolicy::HasQuota(Client* client, bool is_create) {
if (!is_create) {
return false; // no quota enforcement on read requests yet
}
return per_client_cache_.find(client) != per_client_cache_.end();
}
void QuotaAwarePolicy::ObjectCreated(const ObjectID& object_id, Client* client,
bool is_create) {
if (HasQuota(client, is_create)) {
per_client_cache_[client]->Add(object_id, GetObjectSize(object_id));
owned_by_client_[object_id] = client;
} else {
EvictionPolicy::ObjectCreated(object_id, client, is_create);
}
}
bool QuotaAwarePolicy::SetClientQuota(Client* client, int64_t output_memory_quota) {
if (per_client_cache_.find(client) != per_client_cache_.end()) {
ARROW_LOG(WARNING) << "Cannot change the client quota once set";
return false;
}
if (cache_.Capacity() - output_memory_quota <
cache_.OriginalCapacity() * kGlobalLruReserveFraction) {
ARROW_LOG(WARNING) << "Not enough memory to set client quota: " << DebugString();
return false;
}
// those objects will be lazily evicted on the next call
cache_.AdjustCapacity(-output_memory_quota);
per_client_cache_[client] =
std::unique_ptr<LRUCache>(new LRUCache(client->name, output_memory_quota));
return true;
}
bool QuotaAwarePolicy::EnforcePerClientQuota(Client* client, int64_t size, bool is_create,
std::vector<ObjectID>* objects_to_evict) {
if (!HasQuota(client, is_create)) {
return true;
}
auto& client_cache = per_client_cache_[client];
if (size > client_cache->Capacity()) {
ARROW_LOG(WARNING) << "object too large (" << size
<< " bytes) to fit in client quota " << client_cache->Capacity()
<< " " << DebugString();
return false;
}
if (client_cache->RemainingCapacity() >= size) {
return true;
}
int64_t space_to_free = size - client_cache->RemainingCapacity();
if (space_to_free > 0) {
std::vector<ObjectID> candidates;
client_cache->ChooseObjectsToEvict(space_to_free, &candidates);
for (ObjectID& object_id : candidates) {
if (shared_for_read_.count(object_id)) {
// Pinned so we can't evict it, so demote the object to global LRU instead.
// We an do this by simply removing it from all data structures, so that
// the next EndObjectAccess() will add it back to global LRU.
shared_for_read_.erase(object_id);
} else {
objects_to_evict->push_back(object_id);
}
owned_by_client_.erase(object_id);
client_cache->Remove(object_id);
}
}
return true;
}
void QuotaAwarePolicy::BeginObjectAccess(const ObjectID& object_id) {
if (owned_by_client_.find(object_id) != owned_by_client_.end()) {
shared_for_read_.insert(object_id);
pinned_memory_bytes_ += GetObjectSize(object_id);
return;
}
EvictionPolicy::BeginObjectAccess(object_id);
}
void QuotaAwarePolicy::EndObjectAccess(const ObjectID& object_id) {
if (owned_by_client_.find(object_id) != owned_by_client_.end()) {
shared_for_read_.erase(object_id);
pinned_memory_bytes_ -= GetObjectSize(object_id);
return;
}
EvictionPolicy::EndObjectAccess(object_id);
}
void QuotaAwarePolicy::RemoveObject(const ObjectID& object_id) {
if (owned_by_client_.find(object_id) != owned_by_client_.end()) {
per_client_cache_[owned_by_client_[object_id]]->Remove(object_id);
owned_by_client_.erase(object_id);
shared_for_read_.erase(object_id);
return;
}
EvictionPolicy::RemoveObject(object_id);
}
void QuotaAwarePolicy::RefreshObjects(const std::vector<ObjectID>& object_ids) {
for (const auto& object_id : object_ids) {
if (owned_by_client_.find(object_id) != owned_by_client_.end()) {
int64_t size = per_client_cache_[owned_by_client_[object_id]]->Remove(object_id);
per_client_cache_[owned_by_client_[object_id]]->Add(object_id, size);
}
}
EvictionPolicy::RefreshObjects(object_ids);
}
void QuotaAwarePolicy::ClientDisconnected(Client* client) {
if (per_client_cache_.find(client) == per_client_cache_.end()) {
return;
}
// return capacity back to global LRU
cache_.AdjustCapacity(per_client_cache_[client]->Capacity());
// clean up any entries used to track this client's quota usage
per_client_cache_[client]->Foreach([this](const ObjectID& obj) {
if (!shared_for_read_.count(obj)) {
// only add it to the global LRU if we have it in pinned mode
// otherwise, EndObjectAccess will add it later
cache_.Add(obj, GetObjectSize(obj));
}
owned_by_client_.erase(obj);
shared_for_read_.erase(obj);
});
per_client_cache_.erase(client);
}
std::string QuotaAwarePolicy::DebugString() const {
std::stringstream result;
result << "num clients with quota: " << per_client_cache_.size();
result << "\nquota map size: " << owned_by_client_.size();
result << "\npinned quota map size: " << shared_for_read_.size();
result << "\nallocated bytes: " << PlasmaAllocator::Allocated();
result << "\nallocation limit: " << PlasmaAllocator::GetFootprintLimit();
result << "\npinned bytes: " << pinned_memory_bytes_;
result << cache_.DebugString();
for (const auto& pair : per_client_cache_) {
result << pair.second->DebugString();
}
return result.str();
}
} // namespace plasma

View file

@ -0,0 +1,91 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_QUOTA_AWARE_POLICY_H
#define PLASMA_QUOTA_AWARE_POLICY_H
#include <list>
#include <memory>
#include <string>
#include <unordered_map>
#include <unordered_set>
#include <utility>
#include <vector>
#include "plasma/common.h"
#include "plasma/eviction_policy.h"
#include "plasma/plasma.h"
namespace plasma {
/// Reserve this fraction of memory for shared usage. Attempts to set client
/// quotas that would cause the global LRU memory fraction to fall below this
/// value will be rejected.
constexpr double kGlobalLruReserveFraction = 0.3;
/// Extends the basic eviction policy to implement per-client memory quotas.
/// This effectively gives each client its own LRU queue, which caps its
/// memory usage and protects this memory from being evicted by other clients.
///
/// The quotas are enforced when objects are first created, by evicting the
/// necessary number of objects from the client's own LRU queue to cap its
/// memory usage. Once that is done, allocation is handled by the normal
/// eviction policy. This may result in the eviction of objects from the
/// global LRU queue, if not enough memory can be allocated even after the
/// evictions from the client's own LRU queue.
///
/// Some special cases:
/// - When a pinned object is "evicted" from a per-client queue, it is
/// instead transferred into the global LRU queue.
/// - When a client disconnects, its LRU queue is merged into the head of the
/// global LRU queue.
class QuotaAwarePolicy : public EvictionPolicy {
public:
/// Construct a quota-aware eviction policy.
///
/// \param store_info Information about the Plasma store that is exposed
/// to the eviction policy.
/// \param max_size Max size in bytes total of objects to store.
explicit QuotaAwarePolicy(PlasmaStoreInfo* store_info, int64_t max_size);
void ObjectCreated(const ObjectID& object_id, Client* client, bool is_create) override;
bool SetClientQuota(Client* client, int64_t output_memory_quota) override;
bool EnforcePerClientQuota(Client* client, int64_t size, bool is_create,
std::vector<ObjectID>* objects_to_evict) override;
void ClientDisconnected(Client* client) override;
void BeginObjectAccess(const ObjectID& object_id) override;
void EndObjectAccess(const ObjectID& object_id) override;
void RemoveObject(const ObjectID& object_id) override;
void RefreshObjects(const std::vector<ObjectID>& object_ids) override;
std::string DebugString() const override;
private:
/// Returns whether we are enforcing memory quotas for an operation.
bool HasQuota(Client* client, bool is_create);
/// Per-client LRU caches, if quota is enabled.
std::unordered_map<Client*, std::unique_ptr<LRUCache>> per_client_cache_;
/// Tracks which client created which object. This only applies to clients
/// that have a memory quota set.
std::unordered_map<ObjectID, Client*> owned_by_client_;
/// Tracks which objects are mapped for read and hence can't be evicted.
/// However these objects are still tracked within the client caches.
std::unordered_set<ObjectID> shared_for_read_;
};
} // namespace plasma
#endif // PLASMA_EVICTION_POLICY_H

1330
src/ray/plasma/store.cc Normal file

File diff suppressed because it is too large Load diff

250
src/ray/plasma/store.h Normal file
View file

@ -0,0 +1,250 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_STORE_H
#define PLASMA_STORE_H
#include <deque>
#include <memory>
#include <string>
#include <unordered_map>
#include <unordered_set>
#include <vector>
#include "plasma/common.h"
#include "plasma/events.h"
#include "plasma/external_store.h"
#include "plasma/plasma.h"
#include "plasma/protocol.h"
#include "plasma/quota_aware_policy.h"
namespace arrow {
class Status;
} // namespace arrow
namespace plasma {
namespace flatbuf {
struct ObjectInfoT;
enum class PlasmaError;
} // namespace flatbuf
using flatbuf::ObjectInfoT;
using flatbuf::PlasmaError;
struct GetRequest;
struct NotificationQueue {
/// The object notifications for clients. We notify the client about the
/// objects in the order that the objects were sealed or deleted.
std::deque<std::unique_ptr<uint8_t[]>> object_notifications;
};
class PlasmaStore {
public:
using NotificationMap = std::unordered_map<int, NotificationQueue>;
// TODO: PascalCase PlasmaStore methods.
PlasmaStore(EventLoop* loop, std::string directory, bool hugepages_enabled,
const std::string& socket_name,
std::shared_ptr<ExternalStore> external_store);
~PlasmaStore();
/// Get a const pointer to the internal PlasmaStoreInfo object.
const PlasmaStoreInfo* GetPlasmaStoreInfo();
/// Create a new object. The client must do a call to release_object to tell
/// the store when it is done with the object.
///
/// \param object_id Object ID of the object to be created.
/// \param evict_if_full If this is true, then when the object store is full,
/// try to evict objects that are not currently referenced before
/// creating the object. Else, do not evict any objects and
/// immediately return an PlasmaError::OutOfMemory.
/// \param data_size Size in bytes of the object to be created.
/// \param metadata_size Size in bytes of the object metadata.
/// \param device_num The number of the device where the object is being
/// created.
/// device_num = 0 corresponds to the host,
/// device_num = 1 corresponds to GPU0,
/// device_num = 2 corresponds to GPU1, etc.
/// \param client The client that created the object.
/// \param result The object that has been created.
/// \return One of the following error codes:
/// - PlasmaError::OK, if the object was created successfully.
/// - PlasmaError::ObjectExists, if an object with this ID is already
/// present in the store. In this case, the client should not call
/// plasma_release.
/// - PlasmaError::OutOfMemory, if the store is out of memory and
/// cannot create the object. In this case, the client should not call
/// plasma_release.
PlasmaError CreateObject(const ObjectID& object_id, bool evict_if_full,
int64_t data_size, int64_t metadata_size, int device_num,
Client* client, PlasmaObject* result);
/// Abort a created but unsealed object. If the client is not the
/// creator, then the abort will fail.
///
/// \param object_id Object ID of the object to be aborted.
/// \param client The client who created the object. If this does not
/// match the creator of the object, then the abort will fail.
/// \return 1 if the abort succeeds, else 0.
int AbortObject(const ObjectID& object_id, Client* client);
/// Delete a specific object by object_id that have been created in the hash table.
///
/// \param object_id Object ID of the object to be deleted.
/// \return One of the following error codes:
/// - PlasmaError::OK, if the object was delete successfully.
/// - PlasmaError::ObjectNonexistent, if ths object isn't existed.
/// - PlasmaError::ObjectInUse, if the object is in use.
PlasmaError DeleteObject(ObjectID& object_id);
/// Evict objects returned by the eviction policy.
///
/// \param object_ids Object IDs of the objects to be evicted.
void EvictObjects(const std::vector<ObjectID>& object_ids);
/// Process a get request from a client. This method assumes that we will
/// eventually have these objects sealed. If one of the objects has not yet
/// been sealed, the client that requested the object will be notified when it
/// is sealed.
///
/// For each object, the client must do a call to release_object to tell the
/// store when it is done with the object.
///
/// \param client The client making this request.
/// \param object_ids Object IDs of the objects to be gotten.
/// \param timeout_ms The timeout for the get request in milliseconds.
void ProcessGetRequest(Client* client, const std::vector<ObjectID>& object_ids,
int64_t timeout_ms);
/// Seal a vector of objects. The objects are now immutable and can be accessed with
/// get.
///
/// \param object_ids The vector of Object IDs of the objects to be sealed.
/// \param digests The vector of digests of the objects. This is used to tell if two
/// objects with the same object ID are the same.
void SealObjects(const std::vector<ObjectID>& object_ids,
const std::vector<std::string>& digests);
/// Check if the plasma store contains an object:
///
/// \param object_id Object ID that will be checked.
/// \return OBJECT_FOUND if the object is in the store, OBJECT_NOT_FOUND if
/// not
ObjectStatus ContainsObject(const ObjectID& object_id);
/// Record the fact that a particular client is no longer using an object.
///
/// \param object_id The object ID of the object that is being released.
/// \param client The client making this request.
void ReleaseObject(const ObjectID& object_id, Client* client);
/// Subscribe a file descriptor to updates about new sealed objects.
///
/// \param client The client making this request.
void SubscribeToUpdates(Client* client);
/// Connect a new client to the PlasmaStore.
///
/// \param listener_sock The socket that is listening to incoming connections.
void ConnectClient(int listener_sock);
/// Disconnect a client from the PlasmaStore.
///
/// \param client_fd The client file descriptor that is disconnected.
void DisconnectClient(int client_fd);
NotificationMap::iterator SendNotifications(NotificationMap::iterator it);
arrow::Status ProcessMessage(Client* client);
private:
void PushNotification(ObjectInfoT* object_notification);
void PushNotifications(std::vector<ObjectInfoT>& object_notifications);
void PushNotification(ObjectInfoT* object_notification, int client_fd);
void AddToClientObjectIds(const ObjectID& object_id, ObjectTableEntry* entry,
Client* client);
/// Remove a GetRequest and clean up the relevant data structures.
///
/// \param get_request The GetRequest to remove.
void RemoveGetRequest(GetRequest* get_request);
/// Remove all of the GetRequests for a given client.
///
/// \param client The client whose GetRequests should be removed.
void RemoveGetRequestsForClient(Client* client);
void ReturnFromGet(GetRequest* get_req);
void UpdateObjectGetRequests(const ObjectID& object_id);
int RemoveFromClientObjectIds(const ObjectID& object_id, ObjectTableEntry* entry,
Client* client);
void EraseFromObjectTable(const ObjectID& object_id);
uint8_t* AllocateMemory(size_t size, bool evict_if_full, int* fd, int64_t* map_size,
ptrdiff_t* offset, Client* client, bool is_create);
#ifdef PLASMA_CUDA
Status AllocateCudaMemory(int device_num, int64_t size, uint8_t** out_pointer,
std::shared_ptr<CudaIpcMemHandle>* out_ipc_handle);
Status FreeCudaMemory(int device_num, int64_t size, uint8_t* out_pointer);
#endif
/// Event loop of the plasma store.
EventLoop* loop_;
/// The plasma store information, including the object tables, that is exposed
/// to the eviction policy.
PlasmaStoreInfo store_info_;
/// The state that is managed by the eviction policy.
QuotaAwarePolicy eviction_policy_;
/// Input buffer. This is allocated only once to avoid mallocs for every
/// call to process_message.
std::vector<uint8_t> input_buffer_;
/// A hash table mapping object IDs to a vector of the get requests that are
/// waiting for the object to arrive.
std::unordered_map<ObjectID, std::vector<GetRequest*>> object_get_requests_;
/// The pending notifications that have not been sent to subscribers because
/// the socket send buffers were full. This is a hash table from client file
/// descriptor to an array of object_ids to send to that client.
/// TODO(pcm): Consider putting this into the Client data structure and
/// reorganize the code slightly.
NotificationMap pending_notifications_;
std::unordered_map<int, std::unique_ptr<Client>> connected_clients_;
std::unordered_set<ObjectID> deletion_cache_;
/// Manages worker threads for handling asynchronous/multi-threaded requests
/// for reading/writing data to/from external store.
std::shared_ptr<ExternalStore> external_store_;
#ifdef PLASMA_CUDA
arrow::cuda::CudaDeviceManager* manager_;
#endif
};
} // namespace plasma
#endif // PLASMA_STORE_H

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,143 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include <assert.h>
#include <signal.h>
#include <stdlib.h>
#include <sys/time.h>
#include <sys/types.h>
#include <unistd.h>
#include <memory>
#include <thread>
#include <gtest/gtest.h>
#include "arrow/testing/gtest_util.h"
#include "arrow/util/io_util.h"
#include "plasma/client.h"
#include "plasma/common.h"
#include "plasma/external_store.h"
#include "plasma/plasma.h"
#include "plasma/protocol.h"
#include "plasma/test_util.h"
namespace plasma {
using arrow::internal::TemporaryDir;
std::string external_test_executable; // NOLINT
void AssertObjectBufferEqual(const ObjectBuffer& object_buffer,
const std::string& metadata, const std::string& data) {
arrow::AssertBufferEqual(*object_buffer.metadata, metadata);
arrow::AssertBufferEqual(*object_buffer.data, data);
}
class TestPlasmaStoreWithExternal : public ::testing::Test {
public:
// TODO(pcm): At the moment, stdout of the test gets mixed up with
// stdout of the object store. Consider changing that.
void SetUp() override {
ASSERT_OK_AND_ASSIGN(temp_dir_, TemporaryDir::Make("ext-test-"));
store_socket_name_ = temp_dir_->path().ToString() + "store";
std::string plasma_directory =
external_test_executable.substr(0, external_test_executable.find_last_of('/'));
std::string plasma_command = plasma_directory +
"/plasma-store-server -m 1024000 -e " +
"hashtable://test -s " + store_socket_name_ +
" 1> /tmp/log.stdout 2> /tmp/log.stderr & " +
"echo $! > " + store_socket_name_ + ".pid";
PLASMA_CHECK_SYSTEM(system(plasma_command.c_str()));
ARROW_CHECK_OK(client_.Connect(store_socket_name_, ""));
}
void TearDown() override {
ARROW_CHECK_OK(client_.Disconnect());
// Kill plasma_store process that we started
#ifdef COVERAGE_BUILD
// Ask plasma_store to exit gracefully and give it time to write out
// coverage files
std::string plasma_term_command =
"kill -TERM `cat " + store_socket_name_ + ".pid` || exit 0";
PLASMA_CHECK_SYSTEM(system(plasma_term_command.c_str()));
std::this_thread::sleep_for(std::chrono::milliseconds(200));
#endif
std::string plasma_kill_command =
"kill -KILL `cat " + store_socket_name_ + ".pid` || exit 0";
PLASMA_CHECK_SYSTEM(system(plasma_kill_command.c_str()));
}
protected:
PlasmaClient client_;
std::unique_ptr<TemporaryDir> temp_dir_;
std::string store_socket_name_;
};
TEST_F(TestPlasmaStoreWithExternal, EvictionTest) {
std::vector<ObjectID> object_ids;
std::string data(100 * 1024, 'x');
std::string metadata;
for (int i = 0; i < 20; i++) {
ObjectID object_id = random_object_id();
object_ids.push_back(object_id);
// Test for object non-existence.
bool has_object;
ARROW_CHECK_OK(client_.Contains(object_id, &has_object));
ASSERT_FALSE(has_object);
// Test for the object being in local Plasma store.
// Create and seal the object.
ARROW_CHECK_OK(client_.CreateAndSeal(object_id, data, metadata));
// Test that the client can get the object.
ARROW_CHECK_OK(client_.Contains(object_id, &has_object));
ASSERT_TRUE(has_object);
}
for (int i = 0; i < 20; i++) {
// Since we are accessing objects sequentially, every object we
// access would be a cache "miss" owing to LRU eviction.
// Try and access the object from the plasma store first, and then try
// external store on failure. This should succeed to fetch the object.
// However, it may evict the next few objects.
std::vector<ObjectBuffer> object_buffers;
ARROW_CHECK_OK(client_.Get({object_ids[i]}, -1, &object_buffers));
ASSERT_EQ(object_buffers.size(), 1);
ASSERT_EQ(object_buffers[0].device_num, 0);
ASSERT_TRUE(object_buffers[0].data);
AssertObjectBufferEqual(object_buffers[0], metadata, data);
}
// Make sure we still cannot fetch objects that do not exist
std::vector<ObjectBuffer> object_buffers;
ARROW_CHECK_OK(client_.Get({random_object_id()}, 100, &object_buffers));
ASSERT_EQ(object_buffers.size(), 1);
ASSERT_EQ(object_buffers[0].device_num, 0);
ASSERT_EQ(object_buffers[0].data, nullptr);
ASSERT_EQ(object_buffers[0].metadata, nullptr);
}
} // namespace plasma
int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
plasma::external_test_executable = std::string(argv[0]);
return RUN_ALL_TESTS();
}

View file

@ -0,0 +1,333 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include <sstream>
#include <sys/types.h>
#include <unistd.h>
#include <gtest/gtest.h>
#include "arrow/testing/gtest_util.h"
#include "arrow/util/io_util.h"
#include "plasma/common.h"
#include "plasma/io.h"
#include "plasma/plasma.h"
#include "plasma/protocol.h"
#include "plasma/test_util.h"
namespace fb = plasma::flatbuf;
namespace plasma {
using arrow::internal::TemporaryDir;
/**
* Seek to the beginning of a file and read a message from it.
*
* \param fd File descriptor of the file.
* \param message_type Message type that we expect in the file.
*
* \return Pointer to the content of the message. Needs to be freed by the
* caller.
*/
std::vector<uint8_t> read_message_from_file(int fd, MessageType message_type) {
/* Go to the beginning of the file. */
lseek(fd, 0, SEEK_SET);
MessageType type;
std::vector<uint8_t> data;
Status s = ReadMessage(fd, &type, &data);
DCHECK_OK(s);
DCHECK_EQ(type, message_type);
return data;
}
PlasmaObject random_plasma_object(void) {
unsigned int seed = static_cast<unsigned int>(time(NULL));
int random = rand_r(&seed);
PlasmaObject object = {};
object.store_fd = random + 7;
object.data_offset = random + 1;
object.metadata_offset = random + 2;
object.data_size = random + 3;
object.metadata_size = random + 4;
object.device_num = 0;
return object;
}
class TestPlasmaSerialization : public ::testing::Test {
public:
void SetUp() { ASSERT_OK_AND_ASSIGN(temp_dir_, TemporaryDir::Make("ser-test-")); }
// Create a temporary file.
// A fd is returned which must be closed manually. The file itself
// is deleted at the end of the test.
int CreateTemporaryFile(void) {
char path[1024];
std::stringstream ss;
ss << temp_dir_->path().ToString() << "fileXXXXXX";
strncpy(path, ss.str().c_str(), sizeof(path));
ARROW_LOG(INFO) << "file path: '" << path << "'";
return mkstemp(path);
}
protected:
std::unique_ptr<TemporaryDir> temp_dir_;
};
TEST_F(TestPlasmaSerialization, CreateRequest) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
int64_t data_size1 = 42;
int64_t metadata_size1 = 11;
int device_num1 = 0;
ASSERT_OK(SendCreateRequest(fd, object_id1, /*evict_if_full=*/true, data_size1,
metadata_size1, device_num1));
std::vector<uint8_t> data =
read_message_from_file(fd, MessageType::PlasmaCreateRequest);
ObjectID object_id2;
bool evict_if_full;
int64_t data_size2;
int64_t metadata_size2;
int device_num2;
ASSERT_OK(ReadCreateRequest(data.data(), data.size(), &object_id2, &evict_if_full,
&data_size2, &metadata_size2, &device_num2));
ASSERT_TRUE(evict_if_full);
ASSERT_EQ(data_size1, data_size2);
ASSERT_EQ(metadata_size1, metadata_size2);
ASSERT_EQ(object_id1, object_id2);
ASSERT_EQ(device_num1, device_num2);
close(fd);
}
TEST_F(TestPlasmaSerialization, CreateReply) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
PlasmaObject object1 = random_plasma_object();
int64_t mmap_size1 = 1000000;
ASSERT_OK(SendCreateReply(fd, object_id1, &object1, PlasmaError::OK, mmap_size1));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaCreateReply);
ObjectID object_id2;
PlasmaObject object2 = {};
int store_fd;
int64_t mmap_size2;
ASSERT_OK(ReadCreateReply(data.data(), data.size(), &object_id2, &object2, &store_fd,
&mmap_size2));
ASSERT_EQ(object_id1, object_id2);
ASSERT_EQ(object1.store_fd, store_fd);
ASSERT_EQ(mmap_size1, mmap_size2);
ASSERT_EQ(memcmp(&object1, &object2, sizeof(object1)), 0);
close(fd);
}
TEST_F(TestPlasmaSerialization, SealRequest) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
std::string digest1 = std::string(kDigestSize, 7);
ASSERT_OK(SendSealRequest(fd, object_id1, digest1));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaSealRequest);
ObjectID object_id2;
std::string digest2;
ASSERT_OK(ReadSealRequest(data.data(), data.size(), &object_id2, &digest2));
ASSERT_EQ(object_id1, object_id2);
ASSERT_EQ(memcmp(digest1.data(), digest2.data(), kDigestSize), 0);
close(fd);
}
TEST_F(TestPlasmaSerialization, SealReply) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
ASSERT_OK(SendSealReply(fd, object_id1, PlasmaError::ObjectExists));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaSealReply);
ObjectID object_id2;
Status s = ReadSealReply(data.data(), data.size(), &object_id2);
ASSERT_EQ(object_id1, object_id2);
ASSERT_TRUE(IsPlasmaObjectExists(s));
close(fd);
}
TEST_F(TestPlasmaSerialization, GetRequest) {
int fd = CreateTemporaryFile();
ObjectID object_ids[2];
object_ids[0] = random_object_id();
object_ids[1] = random_object_id();
int64_t timeout_ms = 1234;
ASSERT_OK(SendGetRequest(fd, object_ids, 2, timeout_ms));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaGetRequest);
std::vector<ObjectID> object_ids_return;
int64_t timeout_ms_return;
ASSERT_OK(
ReadGetRequest(data.data(), data.size(), object_ids_return, &timeout_ms_return));
ASSERT_EQ(object_ids[0], object_ids_return[0]);
ASSERT_EQ(object_ids[1], object_ids_return[1]);
ASSERT_EQ(timeout_ms, timeout_ms_return);
close(fd);
}
TEST_F(TestPlasmaSerialization, GetReply) {
int fd = CreateTemporaryFile();
ObjectID object_ids[2];
object_ids[0] = random_object_id();
object_ids[1] = random_object_id();
std::unordered_map<ObjectID, PlasmaObject> plasma_objects;
plasma_objects[object_ids[0]] = random_plasma_object();
plasma_objects[object_ids[1]] = random_plasma_object();
std::vector<int> store_fds = {1, 2, 3};
std::vector<int64_t> mmap_sizes = {100, 200, 300};
ASSERT_OK(SendGetReply(fd, object_ids, plasma_objects, 2, store_fds, mmap_sizes));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaGetReply);
ObjectID object_ids_return[2];
PlasmaObject plasma_objects_return[2];
std::vector<int> store_fds_return;
std::vector<int64_t> mmap_sizes_return;
memset(&plasma_objects_return, 0, sizeof(plasma_objects_return));
ASSERT_OK(ReadGetReply(data.data(), data.size(), object_ids_return,
&plasma_objects_return[0], 2, store_fds_return,
mmap_sizes_return));
ASSERT_EQ(object_ids[0], object_ids_return[0]);
ASSERT_EQ(object_ids[1], object_ids_return[1]);
PlasmaObject po, po2;
for (int i = 0; i < 2; ++i) {
po = plasma_objects[object_ids[i]];
po2 = plasma_objects_return[i];
ASSERT_EQ(po, po2);
}
ASSERT_TRUE(store_fds == store_fds_return);
ASSERT_TRUE(mmap_sizes == mmap_sizes_return);
close(fd);
}
TEST_F(TestPlasmaSerialization, ReleaseRequest) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
ASSERT_OK(SendReleaseRequest(fd, object_id1));
std::vector<uint8_t> data =
read_message_from_file(fd, MessageType::PlasmaReleaseRequest);
ObjectID object_id2;
ASSERT_OK(ReadReleaseRequest(data.data(), data.size(), &object_id2));
ASSERT_EQ(object_id1, object_id2);
close(fd);
}
TEST_F(TestPlasmaSerialization, ReleaseReply) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
ASSERT_OK(SendReleaseReply(fd, object_id1, PlasmaError::ObjectExists));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaReleaseReply);
ObjectID object_id2;
Status s = ReadReleaseReply(data.data(), data.size(), &object_id2);
ASSERT_EQ(object_id1, object_id2);
ASSERT_TRUE(IsPlasmaObjectExists(s));
close(fd);
}
TEST_F(TestPlasmaSerialization, DeleteRequest) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
ASSERT_OK(SendDeleteRequest(fd, std::vector<ObjectID>{object_id1}));
std::vector<uint8_t> data =
read_message_from_file(fd, MessageType::PlasmaDeleteRequest);
std::vector<ObjectID> object_vec;
ASSERT_OK(ReadDeleteRequest(data.data(), data.size(), &object_vec));
ASSERT_EQ(object_vec.size(), 1);
ASSERT_EQ(object_id1, object_vec[0]);
close(fd);
}
TEST_F(TestPlasmaSerialization, DeleteReply) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
PlasmaError error1 = PlasmaError::ObjectExists;
ASSERT_OK(SendDeleteReply(fd, std::vector<ObjectID>{object_id1},
std::vector<PlasmaError>{error1}));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaDeleteReply);
std::vector<ObjectID> object_vec;
std::vector<PlasmaError> error_vec;
Status s = ReadDeleteReply(data.data(), data.size(), &object_vec, &error_vec);
ASSERT_EQ(object_vec.size(), 1);
ASSERT_EQ(object_id1, object_vec[0]);
ASSERT_EQ(error_vec.size(), 1);
ASSERT_TRUE(error_vec[0] == PlasmaError::ObjectExists);
ASSERT_TRUE(s.ok());
close(fd);
}
TEST_F(TestPlasmaSerialization, EvictRequest) {
int fd = CreateTemporaryFile();
int64_t num_bytes = 111;
ASSERT_OK(SendEvictRequest(fd, num_bytes));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaEvictRequest);
int64_t num_bytes_received;
ASSERT_OK(ReadEvictRequest(data.data(), data.size(), &num_bytes_received));
ASSERT_EQ(num_bytes, num_bytes_received);
close(fd);
}
TEST_F(TestPlasmaSerialization, EvictReply) {
int fd = CreateTemporaryFile();
int64_t num_bytes = 111;
ASSERT_OK(SendEvictReply(fd, num_bytes));
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaEvictReply);
int64_t num_bytes_received;
ASSERT_OK(ReadEvictReply(data.data(), data.size(), num_bytes_received));
ASSERT_EQ(num_bytes, num_bytes_received);
close(fd);
}
TEST_F(TestPlasmaSerialization, DataRequest) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
const char* address1 = "address1";
int port1 = 12345;
ASSERT_OK(SendDataRequest(fd, object_id1, address1, port1));
/* Reading message back. */
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaDataRequest);
ObjectID object_id2;
char* address2;
int port2;
ASSERT_OK(ReadDataRequest(data.data(), data.size(), &object_id2, &address2, &port2));
ASSERT_EQ(object_id1, object_id2);
ASSERT_EQ(strcmp(address1, address2), 0);
ASSERT_EQ(port1, port2);
free(address2);
close(fd);
}
TEST_F(TestPlasmaSerialization, DataReply) {
int fd = CreateTemporaryFile();
ObjectID object_id1 = random_object_id();
int64_t object_size1 = 146;
int64_t metadata_size1 = 198;
ASSERT_OK(SendDataReply(fd, object_id1, object_size1, metadata_size1));
/* Reading message back. */
std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaDataReply);
ObjectID object_id2;
int64_t object_size2;
int64_t metadata_size2;
ASSERT_OK(ReadDataReply(data.data(), data.size(), &object_id2, &object_size2,
&metadata_size2));
ASSERT_EQ(object_id1, object_id2);
ASSERT_EQ(object_size1, object_size2);
ASSERT_EQ(metadata_size1, metadata_size2);
}
} // namespace plasma

View file

@ -0,0 +1,49 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#ifndef PLASMA_TEST_UTIL_H
#define PLASMA_TEST_UTIL_H
#include <algorithm>
#include <limits>
#include <random>
#include "plasma/common.h"
namespace plasma {
ObjectID random_object_id() {
static uint32_t random_seed = 0;
std::mt19937 gen(random_seed++);
std::uniform_int_distribution<uint32_t> d(0, std::numeric_limits<uint8_t>::max());
ObjectID result;
uint8_t* data = result.mutable_data();
std::generate(data, data + kUniqueIDSize,
[&d, &gen] { return static_cast<uint8_t>(d(gen)); });
return result;
}
#define PLASMA_CHECK_SYSTEM(expr) \
do { \
int status__ = (expr); \
EXPECT_TRUE(WIFEXITED(status__)); \
EXPECT_EQ(WEXITSTATUS(status__), 0); \
} while (false);
} // namespace plasma
#endif // PLASMA_TEST_UTIL_H

465
src/ray/plasma/thirdparty/ae/ae.c vendored Normal file
View file

@ -0,0 +1,465 @@
/* A simple event-driven programming library. Originally I wrote this code
* for the Jim's event-loop (Jim is a Tcl interpreter) but later translated
* it in form of a library for easy reuse.
*
* Copyright (c) 2006-2010, Salvatore Sanfilippo <antirez at gmail dot com>
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <stdio.h>
#include <sys/time.h>
#include <sys/types.h>
#include <unistd.h>
#include <stdlib.h>
#include <poll.h>
#include <string.h>
#include <time.h>
#include <errno.h>
#include "plasma/thirdparty/ae/ae.h"
#include "plasma/thirdparty/ae/zmalloc.h"
#include "plasma/thirdparty/ae/config.h"
/* Include the best multiplexing layer supported by this system.
* The following should be ordered by performances, descending. */
#ifdef HAVE_EVPORT
#include "plasma/thirdparty/ae/ae_evport.c"
#else
#ifdef HAVE_EPOLL
#include "plasma/thirdparty/ae/ae_epoll.c"
#else
#ifdef HAVE_KQUEUE
#include "plasma/thirdparty/ae/ae_kqueue.c"
#else
#include "plasma/thirdparty/ae/ae_select.c"
#endif
#endif
#endif
aeEventLoop *aeCreateEventLoop(int setsize) {
aeEventLoop *eventLoop;
int i;
if ((eventLoop = zmalloc(sizeof(*eventLoop))) == NULL) goto err;
eventLoop->events = zmalloc(sizeof(aeFileEvent)*setsize);
eventLoop->fired = zmalloc(sizeof(aeFiredEvent)*setsize);
if (eventLoop->events == NULL || eventLoop->fired == NULL) goto err;
eventLoop->setsize = setsize;
eventLoop->lastTime = time(NULL);
eventLoop->timeEventHead = NULL;
eventLoop->timeEventNextId = 0;
eventLoop->stop = 0;
eventLoop->maxfd = -1;
eventLoop->beforesleep = NULL;
if (aeApiCreate(eventLoop) == -1) goto err;
/* Events with mask == AE_NONE are not set. So let's initialize the
* vector with it. */
for (i = 0; i < setsize; i++)
eventLoop->events[i].mask = AE_NONE;
return eventLoop;
err:
if (eventLoop) {
zfree(eventLoop->events);
zfree(eventLoop->fired);
zfree(eventLoop);
}
return NULL;
}
/* Return the current set size. */
int aeGetSetSize(aeEventLoop *eventLoop) {
return eventLoop->setsize;
}
/* Resize the maximum set size of the event loop.
* If the requested set size is smaller than the current set size, but
* there is already a file descriptor in use that is >= the requested
* set size minus one, AE_ERR is returned and the operation is not
* performed at all.
*
* Otherwise AE_OK is returned and the operation is successful. */
int aeResizeSetSize(aeEventLoop *eventLoop, int setsize) {
int i;
if (setsize == eventLoop->setsize) return AE_OK;
if (eventLoop->maxfd >= setsize) return AE_ERR;
if (aeApiResize(eventLoop,setsize) == -1) return AE_ERR;
eventLoop->events = zrealloc(eventLoop->events,sizeof(aeFileEvent)*setsize);
eventLoop->fired = zrealloc(eventLoop->fired,sizeof(aeFiredEvent)*setsize);
eventLoop->setsize = setsize;
/* Make sure that if we created new slots, they are initialized with
* an AE_NONE mask. */
for (i = eventLoop->maxfd+1; i < setsize; i++)
eventLoop->events[i].mask = AE_NONE;
return AE_OK;
}
void aeDeleteEventLoop(aeEventLoop *eventLoop) {
aeApiFree(eventLoop);
zfree(eventLoop->events);
zfree(eventLoop->fired);
zfree(eventLoop);
}
void aeStop(aeEventLoop *eventLoop) {
eventLoop->stop = 1;
}
int aeCreateFileEvent(aeEventLoop *eventLoop, int fd, int mask,
aeFileProc *proc, void *clientData)
{
if (fd >= eventLoop->setsize) {
errno = ERANGE;
return AE_ERR;
}
aeFileEvent *fe = &eventLoop->events[fd];
if (aeApiAddEvent(eventLoop, fd, mask) == -1)
return AE_ERR;
fe->mask |= mask;
if (mask & AE_READABLE) fe->rfileProc = proc;
if (mask & AE_WRITABLE) fe->wfileProc = proc;
fe->clientData = clientData;
if (fd > eventLoop->maxfd)
eventLoop->maxfd = fd;
return AE_OK;
}
void aeDeleteFileEvent(aeEventLoop *eventLoop, int fd, int mask)
{
if (fd >= eventLoop->setsize) return;
aeFileEvent *fe = &eventLoop->events[fd];
if (fe->mask == AE_NONE) return;
aeApiDelEvent(eventLoop, fd, mask);
fe->mask = fe->mask & (~mask);
if (fd == eventLoop->maxfd && fe->mask == AE_NONE) {
/* Update the max fd */
int j;
for (j = eventLoop->maxfd-1; j >= 0; j--)
if (eventLoop->events[j].mask != AE_NONE) break;
eventLoop->maxfd = j;
}
}
int aeGetFileEvents(aeEventLoop *eventLoop, int fd) {
if (fd >= eventLoop->setsize) return 0;
aeFileEvent *fe = &eventLoop->events[fd];
return fe->mask;
}
static void aeGetTime(long *seconds, long *milliseconds)
{
struct timeval tv;
gettimeofday(&tv, NULL);
*seconds = tv.tv_sec;
*milliseconds = tv.tv_usec/1000;
}
static void aeAddMillisecondsToNow(long long milliseconds, long *sec, long *ms) {
long cur_sec, cur_ms, when_sec, when_ms;
aeGetTime(&cur_sec, &cur_ms);
when_sec = cur_sec + milliseconds/1000;
when_ms = cur_ms + milliseconds%1000;
if (when_ms >= 1000) {
when_sec ++;
when_ms -= 1000;
}
*sec = when_sec;
*ms = when_ms;
}
long long aeCreateTimeEvent(aeEventLoop *eventLoop, long long milliseconds,
aeTimeProc *proc, void *clientData,
aeEventFinalizerProc *finalizerProc)
{
long long id = eventLoop->timeEventNextId++;
aeTimeEvent *te;
te = zmalloc(sizeof(*te));
if (te == NULL) return AE_ERR;
te->id = id;
aeAddMillisecondsToNow(milliseconds,&te->when_sec,&te->when_ms);
te->timeProc = proc;
te->finalizerProc = finalizerProc;
te->clientData = clientData;
te->next = eventLoop->timeEventHead;
eventLoop->timeEventHead = te;
return id;
}
int aeDeleteTimeEvent(aeEventLoop *eventLoop, long long id)
{
aeTimeEvent *te = eventLoop->timeEventHead;
while(te) {
if (te->id == id) {
te->id = AE_DELETED_EVENT_ID;
return AE_OK;
}
te = te->next;
}
return AE_ERR; /* NO event with the specified ID found */
}
/* Search the first timer to fire.
* This operation is useful to know how many time the select can be
* put in sleep without to delay any event.
* If there are no timers NULL is returned.
*
* Note that's O(N) since time events are unsorted.
* Possible optimizations (not needed by Redis so far, but...):
* 1) Insert the event in order, so that the nearest is just the head.
* Much better but still insertion or deletion of timers is O(N).
* 2) Use a skiplist to have this operation as O(1) and insertion as O(log(N)).
*/
static aeTimeEvent *aeSearchNearestTimer(aeEventLoop *eventLoop)
{
aeTimeEvent *te = eventLoop->timeEventHead;
aeTimeEvent *nearest = NULL;
while(te) {
if (!nearest || te->when_sec < nearest->when_sec ||
(te->when_sec == nearest->when_sec &&
te->when_ms < nearest->when_ms))
nearest = te;
te = te->next;
}
return nearest;
}
/* Process time events */
static int processTimeEvents(aeEventLoop *eventLoop) {
int processed = 0;
aeTimeEvent *te, *prev;
long long maxId;
time_t now = time(NULL);
/* If the system clock is moved to the future, and then set back to the
* right value, time events may be delayed in a random way. Often this
* means that scheduled operations will not be performed soon enough.
*
* Here we try to detect system clock skews, and force all the time
* events to be processed ASAP when this happens: the idea is that
* processing events earlier is less dangerous than delaying them
* indefinitely, and practice suggests it is. */
if (now < eventLoop->lastTime) {
te = eventLoop->timeEventHead;
while(te) {
te->when_sec = 0;
te = te->next;
}
}
eventLoop->lastTime = now;
prev = NULL;
te = eventLoop->timeEventHead;
maxId = eventLoop->timeEventNextId-1;
while(te) {
long now_sec, now_ms;
long long id;
/* Remove events scheduled for deletion. */
if (te->id == AE_DELETED_EVENT_ID) {
aeTimeEvent *next = te->next;
if (prev == NULL)
eventLoop->timeEventHead = te->next;
else
prev->next = te->next;
if (te->finalizerProc)
te->finalizerProc(eventLoop, te->clientData);
zfree(te);
te = next;
continue;
}
/* Make sure we don't process time events created by time events in
* this iteration. Note that this check is currently useless: we always
* add new timers on the head, however if we change the implementation
* detail, this check may be useful again: we keep it here for future
* defense. */
if (te->id > maxId) {
te = te->next;
continue;
}
aeGetTime(&now_sec, &now_ms);
if (now_sec > te->when_sec ||
(now_sec == te->when_sec && now_ms >= te->when_ms))
{
int retval;
id = te->id;
retval = te->timeProc(eventLoop, id, te->clientData);
processed++;
if (retval != AE_NOMORE) {
aeAddMillisecondsToNow(retval,&te->when_sec,&te->when_ms);
} else {
te->id = AE_DELETED_EVENT_ID;
}
}
prev = te;
te = te->next;
}
return processed;
}
/* Process every pending time event, then every pending file event
* (that may be registered by time event callbacks just processed).
* Without special flags the function sleeps until some file event
* fires, or when the next time event occurs (if any).
*
* If flags is 0, the function does nothing and returns.
* if flags has AE_ALL_EVENTS set, all the kind of events are processed.
* if flags has AE_FILE_EVENTS set, file events are processed.
* if flags has AE_TIME_EVENTS set, time events are processed.
* if flags has AE_DONT_WAIT set the function returns ASAP until all
* the events that's possible to process without to wait are processed.
*
* The function returns the number of events processed. */
int aeProcessEvents(aeEventLoop *eventLoop, int flags)
{
int processed = 0, numevents;
/* Nothing to do? return ASAP */
if (!(flags & AE_TIME_EVENTS) && !(flags & AE_FILE_EVENTS)) return 0;
/* Note that we want call select() even if there are no
* file events to process as long as we want to process time
* events, in order to sleep until the next time event is ready
* to fire. */
if (eventLoop->maxfd != -1 ||
((flags & AE_TIME_EVENTS) && !(flags & AE_DONT_WAIT))) {
int j;
aeTimeEvent *shortest = NULL;
struct timeval tv, *tvp;
if (flags & AE_TIME_EVENTS && !(flags & AE_DONT_WAIT))
shortest = aeSearchNearestTimer(eventLoop);
if (shortest) {
long now_sec, now_ms;
aeGetTime(&now_sec, &now_ms);
tvp = &tv;
/* How many milliseconds we need to wait for the next
* time event to fire? */
long long ms =
(shortest->when_sec - now_sec)*1000 +
shortest->when_ms - now_ms;
if (ms > 0) {
tvp->tv_sec = ms/1000;
tvp->tv_usec = (ms % 1000)*1000;
} else {
tvp->tv_sec = 0;
tvp->tv_usec = 0;
}
} else {
/* If we have to check for events but need to return
* ASAP because of AE_DONT_WAIT we need to set the timeout
* to zero */
if (flags & AE_DONT_WAIT) {
tv.tv_sec = tv.tv_usec = 0;
tvp = &tv;
} else {
/* Otherwise we can block */
tvp = NULL; /* wait forever */
}
}
numevents = aeApiPoll(eventLoop, tvp);
for (j = 0; j < numevents; j++) {
aeFileEvent *fe = &eventLoop->events[eventLoop->fired[j].fd];
int mask = eventLoop->fired[j].mask;
int fd = eventLoop->fired[j].fd;
int rfired = 0;
/* note the fe->mask & mask & ... code: maybe an already processed
* event removed an element that fired and we still didn't
* processed, so we check if the event is still valid. */
if (fe->mask & mask & AE_READABLE) {
rfired = 1;
fe->rfileProc(eventLoop,fd,fe->clientData,mask);
}
if (fe->mask & mask & AE_WRITABLE) {
if (!rfired || fe->wfileProc != fe->rfileProc)
fe->wfileProc(eventLoop,fd,fe->clientData,mask);
}
processed++;
}
}
/* Check time events */
if (flags & AE_TIME_EVENTS)
processed += processTimeEvents(eventLoop);
return processed; /* return the number of processed file/time events */
}
/* Wait for milliseconds until the given file descriptor becomes
* writable/readable/exception */
int aeWait(int fd, int mask, long long milliseconds) {
struct pollfd pfd;
int retmask = 0, retval;
memset(&pfd, 0, sizeof(pfd));
pfd.fd = fd;
if (mask & AE_READABLE) pfd.events |= POLLIN;
if (mask & AE_WRITABLE) pfd.events |= POLLOUT;
if ((retval = poll(&pfd, 1, milliseconds))== 1) {
if (pfd.revents & POLLIN) retmask |= AE_READABLE;
if (pfd.revents & POLLOUT) retmask |= AE_WRITABLE;
if (pfd.revents & POLLERR) retmask |= AE_WRITABLE;
if (pfd.revents & POLLHUP) retmask |= AE_WRITABLE;
return retmask;
} else {
return retval;
}
}
void aeMain(aeEventLoop *eventLoop) {
eventLoop->stop = 0;
while (!eventLoop->stop) {
if (eventLoop->beforesleep != NULL)
eventLoop->beforesleep(eventLoop);
aeProcessEvents(eventLoop, AE_ALL_EVENTS);
}
}
char *aeGetApiName(void) {
return aeApiName();
}
void aeSetBeforeSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *beforesleep) {
eventLoop->beforesleep = beforesleep;
}

123
src/ray/plasma/thirdparty/ae/ae.h vendored Normal file
View file

@ -0,0 +1,123 @@
/* A simple event-driven programming library. Originally I wrote this code
* for the Jim's event-loop (Jim is a Tcl interpreter) but later translated
* it in form of a library for easy reuse.
*
* Copyright (c) 2006-2012, Salvatore Sanfilippo <antirez at gmail dot com>
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __AE_H__
#define __AE_H__
#include <time.h>
#define AE_OK 0
#define AE_ERR -1
#define AE_NONE 0
#define AE_READABLE 1
#define AE_WRITABLE 2
#define AE_FILE_EVENTS 1
#define AE_TIME_EVENTS 2
#define AE_ALL_EVENTS (AE_FILE_EVENTS|AE_TIME_EVENTS)
#define AE_DONT_WAIT 4
#define AE_NOMORE -1
#define AE_DELETED_EVENT_ID -1
/* Macros */
#define AE_NOTUSED(V) ((void) V)
struct aeEventLoop;
/* Types and data structures */
typedef void aeFileProc(struct aeEventLoop *eventLoop, int fd, void *clientData, int mask);
typedef int aeTimeProc(struct aeEventLoop *eventLoop, long long id, void *clientData);
typedef void aeEventFinalizerProc(struct aeEventLoop *eventLoop, void *clientData);
typedef void aeBeforeSleepProc(struct aeEventLoop *eventLoop);
/* File event structure */
typedef struct aeFileEvent {
int mask; /* one of AE_(READABLE|WRITABLE) */
aeFileProc *rfileProc;
aeFileProc *wfileProc;
void *clientData;
} aeFileEvent;
/* Time event structure */
typedef struct aeTimeEvent {
long long id; /* time event identifier. */
long when_sec; /* seconds */
long when_ms; /* milliseconds */
aeTimeProc *timeProc;
aeEventFinalizerProc *finalizerProc;
void *clientData;
struct aeTimeEvent *next;
} aeTimeEvent;
/* A fired event */
typedef struct aeFiredEvent {
int fd;
int mask;
} aeFiredEvent;
/* State of an event based program */
typedef struct aeEventLoop {
int maxfd; /* highest file descriptor currently registered */
int setsize; /* max number of file descriptors tracked */
long long timeEventNextId;
time_t lastTime; /* Used to detect system clock skew */
aeFileEvent *events; /* Registered events */
aeFiredEvent *fired; /* Fired events */
aeTimeEvent *timeEventHead;
int stop;
void *apidata; /* This is used for polling API specific data */
aeBeforeSleepProc *beforesleep;
} aeEventLoop;
/* Prototypes */
aeEventLoop *aeCreateEventLoop(int setsize);
void aeDeleteEventLoop(aeEventLoop *eventLoop);
void aeStop(aeEventLoop *eventLoop);
int aeCreateFileEvent(aeEventLoop *eventLoop, int fd, int mask,
aeFileProc *proc, void *clientData);
void aeDeleteFileEvent(aeEventLoop *eventLoop, int fd, int mask);
int aeGetFileEvents(aeEventLoop *eventLoop, int fd);
long long aeCreateTimeEvent(aeEventLoop *eventLoop, long long milliseconds,
aeTimeProc *proc, void *clientData,
aeEventFinalizerProc *finalizerProc);
int aeDeleteTimeEvent(aeEventLoop *eventLoop, long long id);
int aeProcessEvents(aeEventLoop *eventLoop, int flags);
int aeWait(int fd, int mask, long long milliseconds);
void aeMain(aeEventLoop *eventLoop);
char *aeGetApiName(void);
void aeSetBeforeSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *beforesleep);
int aeGetSetSize(aeEventLoop *eventLoop);
int aeResizeSetSize(aeEventLoop *eventLoop, int setsize);
#endif

137
src/ray/plasma/thirdparty/ae/ae_epoll.c vendored Normal file
View file

@ -0,0 +1,137 @@
/* Linux epoll(2) based ae.c module
*
* Copyright (c) 2009-2012, Salvatore Sanfilippo <antirez at gmail dot com>
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <sys/epoll.h>
typedef struct aeApiState {
int epfd;
struct epoll_event *events;
} aeApiState;
static int aeApiCreate(aeEventLoop *eventLoop) {
aeApiState *state = zmalloc(sizeof(aeApiState));
if (!state) return -1;
state->events = zmalloc(sizeof(struct epoll_event)*eventLoop->setsize);
if (!state->events) {
zfree(state);
return -1;
}
state->epfd = epoll_create(1024); /* 1024 is just a hint for the kernel */
if (state->epfd == -1) {
zfree(state->events);
zfree(state);
return -1;
}
eventLoop->apidata = state;
return 0;
}
static int aeApiResize(aeEventLoop *eventLoop, int setsize) {
aeApiState *state = eventLoop->apidata;
state->events = zrealloc(state->events, sizeof(struct epoll_event)*setsize);
return 0;
}
static void aeApiFree(aeEventLoop *eventLoop) {
aeApiState *state = eventLoop->apidata;
close(state->epfd);
zfree(state->events);
zfree(state);
}
static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) {
aeApiState *state = eventLoop->apidata;
struct epoll_event ee;
memset(&ee, 0, sizeof(struct epoll_event)); // avoid valgrind warning
/* If the fd was already monitored for some event, we need a MOD
* operation. Otherwise we need an ADD operation. */
int op = eventLoop->events[fd].mask == AE_NONE ?
EPOLL_CTL_ADD : EPOLL_CTL_MOD;
ee.events = 0;
mask |= eventLoop->events[fd].mask; /* Merge old events */
if (mask & AE_READABLE) ee.events |= EPOLLIN;
if (mask & AE_WRITABLE) ee.events |= EPOLLOUT;
ee.data.fd = fd;
if (epoll_ctl(state->epfd,op,fd,&ee) == -1) return -1;
return 0;
}
static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int delmask) {
aeApiState *state = eventLoop->apidata;
struct epoll_event ee;
memset(&ee, 0, sizeof(struct epoll_event)); // avoid valgrind warning
int mask = eventLoop->events[fd].mask & (~delmask);
ee.events = 0;
if (mask & AE_READABLE) ee.events |= EPOLLIN;
if (mask & AE_WRITABLE) ee.events |= EPOLLOUT;
ee.data.fd = fd;
if (mask != AE_NONE) {
epoll_ctl(state->epfd,EPOLL_CTL_MOD,fd,&ee);
} else {
/* Note, Kernel < 2.6.9 requires a non null event pointer even for
* EPOLL_CTL_DEL. */
epoll_ctl(state->epfd,EPOLL_CTL_DEL,fd,&ee);
}
}
static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) {
aeApiState *state = eventLoop->apidata;
int retval, numevents = 0;
retval = epoll_wait(state->epfd,state->events,eventLoop->setsize,
tvp ? (tvp->tv_sec*1000 + tvp->tv_usec/1000) : -1);
if (retval > 0) {
int j;
numevents = retval;
for (j = 0; j < numevents; j++) {
int mask = 0;
struct epoll_event *e = state->events+j;
if (e->events & EPOLLIN) mask |= AE_READABLE;
if (e->events & EPOLLOUT) mask |= AE_WRITABLE;
if (e->events & EPOLLERR) mask |= AE_WRITABLE;
if (e->events & EPOLLHUP) mask |= AE_WRITABLE;
eventLoop->fired[j].fd = e->data.fd;
eventLoop->fired[j].mask = mask;
}
}
return numevents;
}
static char *aeApiName(void) {
return "epoll";
}

320
src/ray/plasma/thirdparty/ae/ae_evport.c vendored Normal file
View file

@ -0,0 +1,320 @@
/* ae.c module for illumos event ports.
*
* Copyright (c) 2012, Joyent, Inc. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <assert.h>
#include <errno.h>
#include <port.h>
#include <poll.h>
#include <sys/types.h>
#include <sys/time.h>
#include <stdio.h>
static int evport_debug = 0;
/*
* This file implements the ae API using event ports, present on Solaris-based
* systems since Solaris 10. Using the event port interface, we associate file
* descriptors with the port. Each association also includes the set of poll(2)
* events that the consumer is interested in (e.g., POLLIN and POLLOUT).
*
* There's one tricky piece to this implementation: when we return events via
* aeApiPoll, the corresponding file descriptors become dissociated from the
* port. This is necessary because poll events are level-triggered, so if the
* fd didn't become dissociated, it would immediately fire another event since
* the underlying state hasn't changed yet. We must re-associate the file
* descriptor, but only after we know that our caller has actually read from it.
* The ae API does not tell us exactly when that happens, but we do know that
* it must happen by the time aeApiPoll is called again. Our solution is to
* keep track of the last fds returned by aeApiPoll and re-associate them next
* time aeApiPoll is invoked.
*
* To summarize, in this module, each fd association is EITHER (a) represented
* only via the in-kernel association OR (b) represented by pending_fds and
* pending_masks. (b) is only true for the last fds we returned from aeApiPoll,
* and only until we enter aeApiPoll again (at which point we restore the
* in-kernel association).
*/
#define MAX_EVENT_BATCHSZ 512
typedef struct aeApiState {
int portfd; /* event port */
int npending; /* # of pending fds */
int pending_fds[MAX_EVENT_BATCHSZ]; /* pending fds */
int pending_masks[MAX_EVENT_BATCHSZ]; /* pending fds' masks */
} aeApiState;
static int aeApiCreate(aeEventLoop *eventLoop) {
int i;
aeApiState *state = zmalloc(sizeof(aeApiState));
if (!state) return -1;
state->portfd = port_create();
if (state->portfd == -1) {
zfree(state);
return -1;
}
state->npending = 0;
for (i = 0; i < MAX_EVENT_BATCHSZ; i++) {
state->pending_fds[i] = -1;
state->pending_masks[i] = AE_NONE;
}
eventLoop->apidata = state;
return 0;
}
static int aeApiResize(aeEventLoop *eventLoop, int setsize) {
/* Nothing to resize here. */
return 0;
}
static void aeApiFree(aeEventLoop *eventLoop) {
aeApiState *state = eventLoop->apidata;
close(state->portfd);
zfree(state);
}
static int aeApiLookupPending(aeApiState *state, int fd) {
int i;
for (i = 0; i < state->npending; i++) {
if (state->pending_fds[i] == fd)
return (i);
}
return (-1);
}
/*
* Helper function to invoke port_associate for the given fd and mask.
*/
static int aeApiAssociate(const char *where, int portfd, int fd, int mask) {
int events = 0;
int rv, err;
if (mask & AE_READABLE)
events |= POLLIN;
if (mask & AE_WRITABLE)
events |= POLLOUT;
if (evport_debug)
fprintf(stderr, "%s: port_associate(%d, 0x%x) = ", where, fd, events);
rv = port_associate(portfd, PORT_SOURCE_FD, fd, events,
(void *)(uintptr_t)mask);
err = errno;
if (evport_debug)
fprintf(stderr, "%d (%s)\n", rv, rv == 0 ? "no error" : strerror(err));
if (rv == -1) {
fprintf(stderr, "%s: port_associate: %s\n", where, strerror(err));
if (err == EAGAIN)
fprintf(stderr, "aeApiAssociate: event port limit exceeded.");
}
return rv;
}
static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) {
aeApiState *state = eventLoop->apidata;
int fullmask, pfd;
if (evport_debug)
fprintf(stderr, "aeApiAddEvent: fd %d mask 0x%x\n", fd, mask);
/*
* Since port_associate's "events" argument replaces any existing events, we
* must be sure to include whatever events are already associated when
* we call port_associate() again.
*/
fullmask = mask | eventLoop->events[fd].mask;
pfd = aeApiLookupPending(state, fd);
if (pfd != -1) {
/*
* This fd was recently returned from aeApiPoll. It should be safe to
* assume that the consumer has processed that poll event, but we play
* it safer by simply updating pending_mask. The fd will be
* re-associated as usual when aeApiPoll is called again.
*/
if (evport_debug)
fprintf(stderr, "aeApiAddEvent: adding to pending fd %d\n", fd);
state->pending_masks[pfd] |= fullmask;
return 0;
}
return (aeApiAssociate("aeApiAddEvent", state->portfd, fd, fullmask));
}
static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int mask) {
aeApiState *state = eventLoop->apidata;
int fullmask, pfd;
if (evport_debug)
fprintf(stderr, "del fd %d mask 0x%x\n", fd, mask);
pfd = aeApiLookupPending(state, fd);
if (pfd != -1) {
if (evport_debug)
fprintf(stderr, "deleting event from pending fd %d\n", fd);
/*
* This fd was just returned from aeApiPoll, so it's not currently
* associated with the port. All we need to do is update
* pending_mask appropriately.
*/
state->pending_masks[pfd] &= ~mask;
if (state->pending_masks[pfd] == AE_NONE)
state->pending_fds[pfd] = -1;
return;
}
/*
* The fd is currently associated with the port. Like with the add case
* above, we must look at the full mask for the file descriptor before
* updating that association. We don't have a good way of knowing what the
* events are without looking into the eventLoop state directly. We rely on
* the fact that our caller has already updated the mask in the eventLoop.
*/
fullmask = eventLoop->events[fd].mask;
if (fullmask == AE_NONE) {
/*
* We're removing *all* events, so use port_dissociate to remove the
* association completely. Failure here indicates a bug.
*/
if (evport_debug)
fprintf(stderr, "aeApiDelEvent: port_dissociate(%d)\n", fd);
if (port_dissociate(state->portfd, PORT_SOURCE_FD, fd) != 0) {
perror("aeApiDelEvent: port_dissociate");
abort(); /* will not return */
}
} else if (aeApiAssociate("aeApiDelEvent", state->portfd, fd,
fullmask) != 0) {
/*
* ENOMEM is a potentially transient condition, but the kernel won't
* generally return it unless things are really bad. EAGAIN indicates
* we've reached a resource limit, for which it doesn't make sense to
* retry (counter-intuitively). All other errors indicate a bug. In any
* of these cases, the best we can do is to abort.
*/
abort(); /* will not return */
}
}
static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) {
aeApiState *state = eventLoop->apidata;
struct timespec timeout, *tsp;
int mask, i;
uint_t nevents;
port_event_t event[MAX_EVENT_BATCHSZ];
/*
* If we've returned fd events before, we must re-associate them with the
* port now, before calling port_get(). See the block comment at the top of
* this file for an explanation of why.
*/
for (i = 0; i < state->npending; i++) {
if (state->pending_fds[i] == -1)
/* This fd has since been deleted. */
continue;
if (aeApiAssociate("aeApiPoll", state->portfd,
state->pending_fds[i], state->pending_masks[i]) != 0) {
/* See aeApiDelEvent for why this case is fatal. */
abort();
}
state->pending_masks[i] = AE_NONE;
state->pending_fds[i] = -1;
}
state->npending = 0;
if (tvp != NULL) {
timeout.tv_sec = tvp->tv_sec;
timeout.tv_nsec = tvp->tv_usec * 1000;
tsp = &timeout;
} else {
tsp = NULL;
}
/*
* port_getn can return with errno == ETIME having returned some events (!).
* So if we get ETIME, we check nevents, too.
*/
nevents = 1;
if (port_getn(state->portfd, event, MAX_EVENT_BATCHSZ, &nevents,
tsp) == -1 && (errno != ETIME || nevents == 0)) {
if (errno == ETIME || errno == EINTR)
return 0;
/* Any other error indicates a bug. */
perror("aeApiPoll: port_get");
abort();
}
state->npending = nevents;
for (i = 0; i < nevents; i++) {
mask = 0;
if (event[i].portev_events & POLLIN)
mask |= AE_READABLE;
if (event[i].portev_events & POLLOUT)
mask |= AE_WRITABLE;
eventLoop->fired[i].fd = event[i].portev_object;
eventLoop->fired[i].mask = mask;
if (evport_debug)
fprintf(stderr, "aeApiPoll: fd %d mask 0x%x\n",
(int)event[i].portev_object, mask);
state->pending_fds[i] = event[i].portev_object;
state->pending_masks[i] = (uintptr_t)event[i].portev_user;
}
return nevents;
}
static char *aeApiName(void) {
return "evport";
}

138
src/ray/plasma/thirdparty/ae/ae_kqueue.c vendored Normal file
View file

@ -0,0 +1,138 @@
/* Kqueue(2)-based ae.c module
*
* Copyright (C) 2009 Harish Mallipeddi - harish.mallipeddi@gmail.com
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <sys/types.h>
#include <sys/event.h>
#include <sys/time.h>
typedef struct aeApiState {
int kqfd;
struct kevent *events;
} aeApiState;
static int aeApiCreate(aeEventLoop *eventLoop) {
aeApiState *state = zmalloc(sizeof(aeApiState));
if (!state) return -1;
state->events = zmalloc(sizeof(struct kevent)*eventLoop->setsize);
if (!state->events) {
zfree(state);
return -1;
}
state->kqfd = kqueue();
if (state->kqfd == -1) {
zfree(state->events);
zfree(state);
return -1;
}
eventLoop->apidata = state;
return 0;
}
static int aeApiResize(aeEventLoop *eventLoop, int setsize) {
aeApiState *state = eventLoop->apidata;
state->events = zrealloc(state->events, sizeof(struct kevent)*setsize);
return 0;
}
static void aeApiFree(aeEventLoop *eventLoop) {
aeApiState *state = eventLoop->apidata;
close(state->kqfd);
zfree(state->events);
zfree(state);
}
static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) {
aeApiState *state = eventLoop->apidata;
struct kevent ke;
if (mask & AE_READABLE) {
EV_SET(&ke, fd, EVFILT_READ, EV_ADD, 0, 0, NULL);
if (kevent(state->kqfd, &ke, 1, NULL, 0, NULL) == -1) return -1;
}
if (mask & AE_WRITABLE) {
EV_SET(&ke, fd, EVFILT_WRITE, EV_ADD, 0, 0, NULL);
if (kevent(state->kqfd, &ke, 1, NULL, 0, NULL) == -1) return -1;
}
return 0;
}
static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int mask) {
aeApiState *state = eventLoop->apidata;
struct kevent ke;
if (mask & AE_READABLE) {
EV_SET(&ke, fd, EVFILT_READ, EV_DELETE, 0, 0, NULL);
kevent(state->kqfd, &ke, 1, NULL, 0, NULL);
}
if (mask & AE_WRITABLE) {
EV_SET(&ke, fd, EVFILT_WRITE, EV_DELETE, 0, 0, NULL);
kevent(state->kqfd, &ke, 1, NULL, 0, NULL);
}
}
static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) {
aeApiState *state = eventLoop->apidata;
int retval, numevents = 0;
if (tvp != NULL) {
struct timespec timeout;
timeout.tv_sec = tvp->tv_sec;
timeout.tv_nsec = tvp->tv_usec * 1000;
retval = kevent(state->kqfd, NULL, 0, state->events, eventLoop->setsize,
&timeout);
} else {
retval = kevent(state->kqfd, NULL, 0, state->events, eventLoop->setsize,
NULL);
}
if (retval > 0) {
int j;
numevents = retval;
for(j = 0; j < numevents; j++) {
int mask = 0;
struct kevent *e = state->events+j;
if (e->filter == EVFILT_READ) mask |= AE_READABLE;
if (e->filter == EVFILT_WRITE) mask |= AE_WRITABLE;
eventLoop->fired[j].fd = e->ident;
eventLoop->fired[j].mask = mask;
}
}
return numevents;
}
static char *aeApiName(void) {
return "kqueue";
}

106
src/ray/plasma/thirdparty/ae/ae_select.c vendored Normal file
View file

@ -0,0 +1,106 @@
/* Select()-based ae.c module.
*
* Copyright (c) 2009-2012, Salvatore Sanfilippo <antirez at gmail dot com>
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <sys/select.h>
#include <string.h>
typedef struct aeApiState {
fd_set rfds, wfds;
/* We need to have a copy of the fd sets as it's not safe to reuse
* FD sets after select(). */
fd_set _rfds, _wfds;
} aeApiState;
static int aeApiCreate(aeEventLoop *eventLoop) {
aeApiState *state = zmalloc(sizeof(aeApiState));
if (!state) return -1;
FD_ZERO(&state->rfds);
FD_ZERO(&state->wfds);
eventLoop->apidata = state;
return 0;
}
static int aeApiResize(aeEventLoop *eventLoop, int setsize) {
/* Just ensure we have enough room in the fd_set type. */
if (setsize >= FD_SETSIZE) return -1;
return 0;
}
static void aeApiFree(aeEventLoop *eventLoop) {
zfree(eventLoop->apidata);
}
static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) {
aeApiState *state = eventLoop->apidata;
if (mask & AE_READABLE) FD_SET(fd,&state->rfds);
if (mask & AE_WRITABLE) FD_SET(fd,&state->wfds);
return 0;
}
static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int mask) {
aeApiState *state = eventLoop->apidata;
if (mask & AE_READABLE) FD_CLR(fd,&state->rfds);
if (mask & AE_WRITABLE) FD_CLR(fd,&state->wfds);
}
static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) {
aeApiState *state = eventLoop->apidata;
int retval, j, numevents = 0;
memcpy(&state->_rfds,&state->rfds,sizeof(fd_set));
memcpy(&state->_wfds,&state->wfds,sizeof(fd_set));
retval = select(eventLoop->maxfd+1,
&state->_rfds,&state->_wfds,NULL,tvp);
if (retval > 0) {
for (j = 0; j <= eventLoop->maxfd; j++) {
int mask = 0;
aeFileEvent *fe = &eventLoop->events[j];
if (fe->mask == AE_NONE) continue;
if (fe->mask & AE_READABLE && FD_ISSET(j,&state->_rfds))
mask |= AE_READABLE;
if (fe->mask & AE_WRITABLE && FD_ISSET(j,&state->_wfds))
mask |= AE_WRITABLE;
eventLoop->fired[numevents].fd = j;
eventLoop->fired[numevents].mask = mask;
numevents++;
}
}
return numevents;
}
static char *aeApiName(void) {
return "select";
}

54
src/ray/plasma/thirdparty/ae/config.h vendored Normal file
View file

@ -0,0 +1,54 @@
/*
* Copyright (c) 2009-2012, Salvatore Sanfilippo <antirez at gmail dot com>
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __CONFIG_H
#define __CONFIG_H
#ifdef __APPLE__
#include <AvailabilityMacros.h>
#endif
/* Test for polling API */
#ifdef __linux__
#define HAVE_EPOLL 1
#endif
#if (defined(__APPLE__) && defined(MAC_OS_X_VERSION_10_6)) || defined(__FreeBSD__) || defined(__OpenBSD__) || defined (__NetBSD__)
#define HAVE_KQUEUE 1
#endif
#ifdef __sun
#include <sys/feature_tests.h>
#ifdef _DTRACE_VERSION
#define HAVE_EVPORT 1
#endif
#endif
#endif

45
src/ray/plasma/thirdparty/ae/zmalloc.h vendored Normal file
View file

@ -0,0 +1,45 @@
/*
* Copyright (c) 2009-2012, Salvatore Sanfilippo <antirez at gmail dot com>
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef _ZMALLOC_H
#define _ZMALLOC_H
#ifndef zmalloc
#define zmalloc malloc
#endif
#ifndef zfree
#define zfree free
#endif
#ifndef zrealloc
#define zrealloc realloc
#endif
#endif /* _ZMALLOC_H */

6296
src/ray/plasma/thirdparty/dlmalloc.c vendored Normal file

File diff suppressed because it is too large Load diff

View file

@ -6,7 +6,7 @@
set -e set -e
set -x set -x
bazel build "//:object_manager_stress_test" "//:object_manager_test" "@plasma//:plasma_store_server" bazel build "//:object_manager_stress_test" "//:object_manager_test" "//:plasma_store_server"
# Get the directory in which this script is executing. # Get the directory in which this script is executing.
SCRIPT_DIR="`dirname \"$0\"`" SCRIPT_DIR="`dirname \"$0\"`"
@ -24,7 +24,7 @@ fi
REDIS_MODULE="./bazel-bin/libray_redis_module.so" REDIS_MODULE="./bazel-bin/libray_redis_module.so"
LOAD_MODULE_ARGS="--loadmodule ${REDIS_MODULE}" LOAD_MODULE_ARGS="--loadmodule ${REDIS_MODULE}"
STORE_EXEC="./bazel-bin/external/plasma/plasma_store_server" STORE_EXEC="./bazel-bin/plasma_store_server"
# Allow cleanup commands to fail. # Allow cleanup commands to fail.
bazel run //:redis-cli -- -p 6379 shutdown || true bazel run //:redis-cli -- -p 6379 shutdown || true

View file

@ -6,7 +6,7 @@
set -e set -e
set -x set -x
bazel build "//:object_manager_stress_test" "//:object_manager_test" "@plasma//:plasma_store_server" bazel build "//:object_manager_stress_test" "//:object_manager_test" "//:plasma_store_server"
# Get the directory in which this script is executing. # Get the directory in which this script is executing.
SCRIPT_DIR="`dirname \"$0\"`" SCRIPT_DIR="`dirname \"$0\"`"
@ -24,7 +24,7 @@ fi
REDIS_MODULE="./bazel-bin/libray_redis_module.so" REDIS_MODULE="./bazel-bin/libray_redis_module.so"
LOAD_MODULE_ARGS="--loadmodule ${REDIS_MODULE}" LOAD_MODULE_ARGS="--loadmodule ${REDIS_MODULE}"
STORE_EXEC="./bazel-bin/external/plasma/plasma_store_server" STORE_EXEC="./bazel-bin/plasma_store_server"
VALGRIND_CMD="valgrind --track-origins=yes --leak-check=full --show-leak-kinds=all --leak-check-heuristics=stdstring --error-exitcode=1" VALGRIND_CMD="valgrind --track-origins=yes --leak-check=full --show-leak-kinds=all --leak-check-heuristics=stdstring --error-exitcode=1"

View file

@ -35,7 +35,7 @@ if [ -z "$RAY_ROOT" ] ; then
exit 1 exit 1
fi fi
bazel build "//:core_worker_test" "//:mock_worker" "//:raylet" "//:gcs_server" "//:libray_redis_module.so" "@plasma//:plasma_store_server" "//:redis-server" "//:redis-cli" bazel build "//:core_worker_test" "//:mock_worker" "//:raylet" "//:gcs_server" "//:libray_redis_module.so" "//:plasma_store_server" "//:redis-server" "//:redis-cli"
bazel build //streaming:streaming_test_worker bazel build //streaming:streaming_test_worker
bazel build //streaming:streaming_queue_tests bazel build //streaming:streaming_queue_tests
@ -47,7 +47,7 @@ fi
REDIS_MODULE="./bazel-bin/libray_redis_module.so" REDIS_MODULE="./bazel-bin/libray_redis_module.so"
REDIS_SERVER_EXEC="./bazel-bin/redis-server" REDIS_SERVER_EXEC="./bazel-bin/redis-server"
STORE_EXEC="./bazel-bin/external/plasma/plasma_store_server" STORE_EXEC="./bazel-bin/plasma_store_server"
REDIS_CLIENT_EXEC="./bazel-bin/redis-cli" REDIS_CLIENT_EXEC="./bazel-bin/redis-cli"
RAYLET_EXEC="./bazel-bin/raylet" RAYLET_EXEC="./bazel-bin/raylet"
STREAMING_TEST_WORKER_EXEC="./bazel-bin/streaming/streaming_test_worker" STREAMING_TEST_WORKER_EXEC="./bazel-bin/streaming/streaming_test_worker"

View file

@ -1,14 +0,0 @@
diff --git cpp/src/plasma/client.cc cpp/src/plasma/client.cc
--- cpp/src/plasma/client.cc
+++ cpp/src/plasma/client.cc
@@ -22,4 +22,0 @@
-#ifdef _WIN32
-#include <Win32_Interop/win32_types.h>
-#endif
-
diff --git cpp/src/plasma/plasma.h cpp/src/plasma/plasma.h
--- cpp/src/plasma/plasma.h
+++ cpp/src/plasma/plasma.h
@@ -28,1 +28,0 @@
-#include <unistd.h> // pid_t
--

View file

@ -1,57 +0,0 @@
diff --git cpp/src/plasma/malloc.cc cpp/src/plasma/malloc.cc
--- cpp/src/plasma/malloc.cc
+++ cpp/src/plasma/malloc.cc
@@ -26,2 +26,0 @@
-#include <sys/mman.h>
-#include <unistd.h>
diff --git cpp/src/plasma/dlmalloc.cc cpp/src/plasma/dlmalloc.cc
--- cpp/src/plasma/dlmalloc.cc
+++ cpp/src/plasma/dlmalloc.cc
@@ -25,2 +25,6 @@
+#ifdef _WIN32
+#include <Windows.h>
+#else
#include <sys/mman.h>
#include <unistd.h>
+#endif
@@ -76,5 +80,8 @@ int create_buffer(int64_t size) {
- if (!CreateFileMapping(INVALID_HANDLE_VALUE, NULL, PAGE_READWRITE,
- (DWORD)((uint64_t)size >> (CHAR_BIT * sizeof(DWORD))),
- (DWORD)(uint64_t)size, NULL)) {
+ HANDLE h = CreateFileMapping(INVALID_HANDLE_VALUE, NULL, PAGE_READWRITE,
+ (DWORD)((uint64_t)size >> (CHAR_BIT * sizeof(DWORD))),
+ (DWORD)(uint64_t)size, NULL);
+ if (h) {
+ fd = fh_open(reinterpret_cast<intptr_t>(h), -1);
+ } else {
fd = -1;
}
@@ -119,9 +126,18 @@
+ void* pointer;
+#ifdef _WIN32
+ pointer = MapViewOfFile(reinterpret_cast<HANDLE>(fh_get(fd)), FILE_MAP_ALL_ACCESS, 0, 0, size);
+ if (pointer == NULL) {
+ ARROW_LOG(ERROR) << "MapViewOfFile failed with error: " << GetLastError();
+ return reinterpret_cast<void*>(-1);
+ }
+#else
- void* pointer = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
+ pointer = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
if (pointer == MAP_FAILED) {
ARROW_LOG(ERROR) << "mmap failed with error: " << std::strerror(errno);
if (errno == ENOMEM && plasma_config->hugepages_enabled) {
ARROW_LOG(ERROR)
<< " (this probably means you have to increase /proc/sys/vm/nr_hugepages)";
}
return pointer;
}
+#endif
@@ -155,1 +169,6 @@
+ int r;
+#ifdef _WIN32
+ r = UnmapViewOfFile(addr) ? 0 : -1;
+#else
- int r = munmap(addr, size);
+ r = munmap(addr, size);
+#endif
--

View file

@ -5,13 +5,4 @@ diff --git cpp/src/arrow/io/mman.h cpp/src/arrow/io/mman.h
+#ifdef _WIN32 +#ifdef _WIN32
+typedef long long off_t; +typedef long long off_t;
+#endif +#endif
diff --git cpp/src/plasma/protocol.cc cpp/src/plasma/protocol.cc
--- cpp/src/plasma/protocol.cc
+++ cpp/src/plasma/protocol.cc
@@ -760,1 +760,5 @@
+#ifdef _WIN32
+ *address = _strdup(message->address()->c_str());
+#else
*address = strdup(message->address()->c_str());
+#endif
-- --

View file

@ -1,10 +0,0 @@
diff --git cpp/src/plasma/store.cc cpp/src/plasma/store.cc
--- cpp/src/plasma/store.cc
+++ cpp/src/plasma/store.cc
@@ -1185,3 +1185,5 @@ void StartServer(char* socket_name, std::string plasma_directory, bool hugepages_enabled,
+#ifndef _WIN32 // TODO(mehrdadn): Is there an equivalent of this we need for Windows?
// Ignore SIGPIPE signals. If we don't do this, then when we attempt to write
// to a client that has already died, the store could die.
signal(SIGPIPE, SIG_IGN);
+#endif
--

View file

@ -1,159 +0,0 @@
diff --git cpp/src/plasma/client.cc cpp/src/plasma/client.cc
--- cpp/src/plasma/client.cc
+++ cpp/src/plasma/client.cc
@@ -28,1 +28,5 @@
+#ifdef _WIN32
+#include <Windows.h>
+#else
#include <sys/mman.h>
+#endif
@@ -33,1 +37,3 @@
+#ifndef _WIN32
#include <unistd.h>
+#endif
@@ -178,6 +184,14 @@
+#ifdef _WIN32
+ pointer_ = reinterpret_cast<uint8_t*>(MapViewOfFile(reinterpret_cast<HANDLE>(fh_get(fd)), FILE_MAP_ALL_ACCESS, 0, 0, length_));
+ // TODO(pcm): Don't fail here, instead return a Status.
+ if (pointer_ == NULL) {
+ ARROW_LOG(FATAL) << "mmap failed";
+ }
+#else
pointer_ = reinterpret_cast<uint8_t*>(
mmap(NULL, length_, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0));
// TODO(pcm): Don't fail here, instead return a Status.
if (pointer_ == MAP_FAILED) {
ARROW_LOG(FATAL) << "mmap failed";
}
+#endif
@@ -189,1 +195,6 @@
+ int r;
+#ifdef _WIN32
+ r = UnmapViewOfFile(pointer_) ? 0 : -1;
+#else
- int r = munmap(pointer_, length_);
+ r = munmap(pointer_, length_);
+#endif
@@ -990,5 +1009,17 @@
+#ifdef _WINSOCKAPI_
+ SOCKET sockets[2] = { INVALID_SOCKET, INVALID_SOCKET };
+ socketpair(AF_INET, SOCK_STREAM, 0, sockets);
+ sock[0] = fh_open(sockets[0], -1);
+ sock[1] = fh_open(sockets[1], -1);
+#else
socketpair(AF_UNIX, SOCK_STREAM, 0, sock);
+#endif
// Make the socket non-blocking.
+#ifdef _WINSOCKAPI_
+ unsigned long value = 1;
+ ARROW_CHECK(ioctlsocket(sock[1], FIONBIO, &value) == 0);
+#else
int flags = fcntl(sock[1], F_GETFL, 0);
ARROW_CHECK(fcntl(sock[1], F_SETFL, flags | O_NONBLOCK) == 0);
+#endif
// Tell the Plasma store about the subscription.
diff --git cpp/src/plasma/fling.cc cpp/src/plasma/fling.cc
--- cpp/src/plasma/fling.cc
+++ cpp/src/plasma/fling.cc
@@ -19,7 +19,14 @@
#include "arrow/util/logging.h"
+#ifdef _WIN32
+#include <ws2tcpip.h> // socklen_t
+#else
+typedef int SOCKET;
+#endif
+
void init_msg(struct msghdr* msg, struct iovec* iov, char* buf, size_t buf_len) {
iov->iov_base = buf;
iov->iov_len = 1;
+ msg->msg_flags = 0;
msg->msg_iov = iov;
@@ -36,3 +43,8 @@
+#ifdef _WIN32
+ SOCKET to_send = fh_get(fd);
+#else
+ SOCKET to_send = fd;
+#endif
- char buf[CMSG_SPACE(sizeof(int))];
+ char buf[CMSG_SPACE(sizeof(to_send))];
- memset(&buf, 0, CMSG_SPACE(sizeof(int)));
+ memset(&buf, 0, sizeof(buf));
@@ -47,7 +59,12 @@
- header->cmsg_len = CMSG_LEN(sizeof(int));
+ header->cmsg_len = CMSG_LEN(sizeof(to_send));
- memcpy(CMSG_DATA(header), reinterpret_cast<void*>(&fd), sizeof(int));
+ memcpy(CMSG_DATA(header), reinterpret_cast<void*>(&to_send), sizeof(to_send));
+#ifdef _WIN32
+ SOCKET sock = fh_get(conn);
+#else
+ SOCKET sock = conn;
+#endif
// Send file descriptor.
while (true) {
- ssize_t r = sendmsg(conn, &msg, 0);
+ ssize_t r = sendmsg(sock, &msg, 0);
if (r < 0) {
@@ -83,6 +100,11 @@
- char buf[CMSG_SPACE(sizeof(int))];
+ char buf[CMSG_SPACE(sizeof(SOCKET))];
init_msg(&msg, &iov, buf, sizeof(buf));
+#ifdef _WIN32
+ SOCKET sock = fh_get(conn);
+#else
+ int sock = conn;
+#endif
while (true) {
- ssize_t r = recvmsg(conn, &msg, 0);
+ ssize_t r = recvmsg(sock, &msg, 0);
if (r == -1) {
@@ -100,18 +122,22 @@
- int found_fd = -1;
+ SOCKET found_fd = -1;
int oh_noes = 0;
for (struct cmsghdr* header = CMSG_FIRSTHDR(&msg); header != NULL;
header = CMSG_NXTHDR(&msg, header))
if (header->cmsg_level == SOL_SOCKET && header->cmsg_type == SCM_RIGHTS) {
ssize_t count = (header->cmsg_len -
(CMSG_DATA(header) - reinterpret_cast<unsigned char*>(header))) /
- sizeof(int);
+ sizeof(SOCKET);
for (int i = 0; i < count; ++i) {
- int fd = (reinterpret_cast<int*>(CMSG_DATA(header)))[i];
+ SOCKET fd = (reinterpret_cast<SOCKET*>(CMSG_DATA(header)))[i];
if (found_fd == -1) {
found_fd = fd;
} else {
+#ifdef _WIN32
+ closesocket(fd) == 0 || ((WSAGetLastError() == WSAENOTSOCK || WSAGetLastError() == WSANOTINITIALISED) && CloseHandle(reinterpret_cast<HANDLE>(fd)));
+#else
close(fd);
+#endif
oh_noes = 1;
}
}
}
@@ -122,8 +148,17 @@
if (oh_noes) {
+#ifdef _WIN32
+ closesocket(found_fd) == 0 || ((WSAGetLastError() == WSAENOTSOCK || WSAGetLastError() == WSANOTINITIALISED) && CloseHandle(reinterpret_cast<HANDLE>(found_fd)));
+#else
close(found_fd);
+#endif
errno = EBADMSG;
return -1;
}
- return found_fd;
+#ifdef _WIN32
+ int to_receive = fh_open(found_fd, -1);
+#else
+ int to_receive = found_fd;
+#endif
+ return to_receive;
}
--

View file

@ -1,190 +0,0 @@
diff --git cpp/src/plasma/io.h cpp/src/plasma/io.h
--- cpp/src/plasma/io.h
+++ cpp/src/plasma/io.h
@@ -30,2 +30,3 @@
#include "arrow/status.h"
+#include "plasma/common.h"
#include "plasma/compat.h"
@@ -57,3 +58,1 @@
-int BindIpcSock(const std::string& pathname, bool shall_listen);
-
-int ConnectIpcSock(const std::string& pathname);
+int ConnectOrListenIpcSock(const std::string& pathname, bool shall_listen);
diff --git cpp/src/plasma/store.cc cpp/src/plasma/store.cc
--- cpp/src/plasma/store.cc
+++ cpp/src/plasma/store.cc
@@ -1150,1 +1150,1 @@
- int socket = BindIpcSock(socket_name, true);
+ int socket = ConnectOrListenIpcSock(socket_name, true);
@@ -1301,1 +1301,5 @@
+#ifdef _WINSOCKAPI_
+ WSADATA wsadata;
+ WSAStartup(MAKEWORD(2, 2), &wsadata);
+#endif
plasma::StartServer(socket_name, plasma_directory, hugepages_enabled, external_store);
@@ -1307,1 +1311,4 @@
+#ifdef _WINSOCKAPI_
+ WSACleanup();
+#endif
return 0;
diff --git cpp/src/plasma/io.cc cpp/src/plasma/io.cc
--- cpp/src/plasma/io.cc
+++ cpp/src/plasma/io.cc
@@ -29,1 +29,5 @@
+#ifndef _WIN32
+#include <arpa/inet.h>
+#include <netinet/in.h>
+#endif
@@ -117,39 +121,79 @@
-int BindIpcSock(const std::string& pathname, bool shall_listen) {
- struct sockaddr_un socket_address;
- int socket_fd = socket(AF_UNIX, SOCK_STREAM, 0);
+int ConnectOrListenIpcSock(const std::string& pathname, bool shall_listen) {
+ union {
+ struct sockaddr addr;
+ struct sockaddr_un un;
+ struct sockaddr_in in;
+ } socket_address;
+ int addrlen;
+ memset(&socket_address, 0, sizeof(socket_address));
+ if (pathname.find("tcp://") == 0) {
+ addrlen = sizeof(socket_address.in);
+ socket_address.in.sin_family = AF_INET;
+ std::string addr = pathname.substr(pathname.find('/') + 2);
+ size_t i = addr.rfind(':'), j;
+ if (i >= addr.size()) {
+ j = i = addr.size();
+ } else {
+ j = i + 1;
+ }
+ socket_address.in.sin_addr.s_addr = inet_addr(addr.substr(0, i).c_str());
+ socket_address.in.sin_port = htons(static_cast<short>(atoi(addr.substr(j).c_str())));
+ if (socket_address.in.sin_addr.s_addr == INADDR_NONE) {
+ ARROW_LOG(ERROR) << "Socket address is not a valid IPv4 address: " << pathname;
+ return -1;
+ }
+ if (socket_address.in.sin_port == htons(0)) {
+ ARROW_LOG(ERROR) << "Socket address is missing a valid port: " << pathname;
+ return -1;
+ }
+ } else {
+ addrlen = sizeof(socket_address.un);
+ socket_address.un.sun_family = AF_UNIX;
+ if (pathname.size() + 1 > sizeof(socket_address.un.sun_path)) {
+ ARROW_LOG(ERROR) << "Socket pathname is too long.";
+ return -1;
+ }
+ strncpy(socket_address.un.sun_path, pathname.c_str(), pathname.size() + 1);
+ }
+
+ int socket_fd = socket(socket_address.addr.sa_family, SOCK_STREAM, 0);
if (socket_fd < 0) {
ARROW_LOG(ERROR) << "socket() failed for pathname " << pathname;
return -1;
}
- // Tell the system to allow the port to be reused.
- int on = 1;
- if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast<char*>(&on),
- sizeof(on)) < 0) {
- ARROW_LOG(ERROR) << "setsockopt failed for pathname " << pathname;
- close(socket_fd);
- return -1;
- }
+ if (shall_listen) {
+ // Tell the system to allow the port to be reused.
+ int on = 1;
+ if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast<char*>(&on),
+ sizeof(on)) < 0) {
+ ARROW_LOG(ERROR) << "setsockopt failed for pathname " << pathname;
+ close(socket_fd);
+ return -1;
+ }
- unlink(pathname.c_str());
- memset(&socket_address, 0, sizeof(socket_address));
- socket_address.sun_family = AF_UNIX;
- if (pathname.size() + 1 > sizeof(socket_address.sun_path)) {
- ARROW_LOG(ERROR) << "Socket pathname is too long.";
- close(socket_fd);
- return -1;
- }
- strncpy(socket_address.sun_path, pathname.c_str(), pathname.size() + 1);
+ if (socket_address.addr.sa_family == AF_UNIX) {
+#ifdef _WIN32
+ _unlink(pathname.c_str());
+#else
+ unlink(pathname.c_str());
+#endif
+ }
+ if (bind(socket_fd, &socket_address.addr, addrlen) != 0) {
+ ARROW_LOG(ERROR) << "Bind failed for pathname " << pathname;
+ close(socket_fd);
+ return -1;
+ }
- if (bind(socket_fd, reinterpret_cast<struct sockaddr*>(&socket_address),
- sizeof(socket_address)) != 0) {
- ARROW_LOG(ERROR) << "Bind failed for pathname " << pathname;
- close(socket_fd);
- return -1;
- }
- if (shall_listen && listen(socket_fd, 128) == -1) {
- ARROW_LOG(ERROR) << "Could not listen to socket " << pathname;
- close(socket_fd);
- return -1;
+ if (listen(socket_fd, 128) == -1) {
+ ARROW_LOG(ERROR) << "Could not listen to socket " << pathname;
+ close(socket_fd);
+ return -1;
+ }
+ } else {
+ if (connect(socket_fd, &socket_address.addr, addrlen) != 0) {
+ close(socket_fd);
+ return -1;
+ }
}
return socket_fd;
}
@@ -166,9 +270,9 @@
- *fd = ConnectIpcSock(pathname);
+ *fd = ConnectOrListenIpcSock(pathname, false);
while (*fd < 0 && num_retries > 0) {
ARROW_LOG(ERROR) << "Connection to IPC socket failed for pathname " << pathname
<< ", retrying " << num_retries << " more times";
// Sleep for timeout milliseconds.
usleep(static_cast<int>(timeout * 1000));
- *fd = ConnectIpcSock(pathname);
+ *fd = ConnectOrListenIpcSock(pathname, false);
--num_retries;
}
@@ -184,28 +228,0 @@
-int ConnectIpcSock(const std::string& pathname) {
- struct sockaddr_un socket_address;
- int socket_fd;
-
- socket_fd = socket(AF_UNIX, SOCK_STREAM, 0);
- if (socket_fd < 0) {
- ARROW_LOG(ERROR) << "socket() failed for pathname " << pathname;
- return -1;
- }
-
- memset(&socket_address, 0, sizeof(socket_address));
- socket_address.sun_family = AF_UNIX;
- if (pathname.size() + 1 > sizeof(socket_address.sun_path)) {
- ARROW_LOG(ERROR) << "Socket pathname is too long.";
- close(socket_fd);
- return -1;
- }
- strncpy(socket_address.sun_path, pathname.c_str(), pathname.size() + 1);
-
- if (connect(socket_fd, reinterpret_cast<struct sockaddr*>(&socket_address),
- sizeof(socket_address)) != 0) {
- close(socket_fd);
- return -1;
- }
-
- return socket_fd;
-}
-
--