Skip to content

Commit

Permalink
Fix incompatibility with most recent version of Redis. (#3379)
Browse files Browse the repository at this point in the history
* Fix incompatibility with most recent version of Redis.

* Fix

* Fixes.
  • Loading branch information
robertnishihara authored and devin-petersohn committed Nov 25, 2018
1 parent 18a8dbf commit 3856533
Show file tree
Hide file tree
Showing 7 changed files with 8 additions and 24 deletions.
2 changes: 0 additions & 2 deletions python/ray/experimental/state.py
Original file line number Diff line number Diff line change
Expand Up @@ -1151,8 +1151,6 @@ def workers(self):
worker_id = binary_to_hex(worker_key[len("Workers:"):])

workers_data[worker_id] = {
"local_scheduler_socket": (decode(
worker_info[b"local_scheduler_socket"])),
"node_ip_address": decode(worker_info[b"node_ip_address"]),
"plasma_store_socket": decode(
worker_info[b"plasma_store_socket"])
Expand Down
2 changes: 1 addition & 1 deletion python/ray/import_thread.py
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ def fetch_and_execute_function_to_run(self, key):
run_on_other_drivers) = self.redis_client.hmget(
key, ["driver_id", "function", "run_on_other_drivers"])

if (run_on_other_drivers == "False"
if (utils.decode(run_on_other_drivers) == "False"
and self.worker.mode == ray.SCRIPT_MODE
and driver_id != self.worker.task_driver_id.id()):
return
Expand Down
1 change: 0 additions & 1 deletion python/ray/services.py
Original file line number Diff line number Diff line change
Expand Up @@ -1162,7 +1162,6 @@ def start_worker(node_ip_address,
sys.executable, "-u", worker_path,
"--node-ip-address=" + node_ip_address,
"--object-store-name=" + object_store_name,
"--local-scheduler-name=" + local_scheduler_name,
"--redis-address=" + str(redis_address),
"--temp-dir=" + get_temp_root()
]
Expand Down
18 changes: 6 additions & 12 deletions python/ray/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -713,7 +713,7 @@ def run_function_on_all_workers(self, function,
"driver_id": self.task_driver_id.id(),
"function_id": function_to_run_id,
"function": pickled_function,
"run_on_other_drivers": run_on_other_drivers
"run_on_other_drivers": str(run_on_other_drivers)
})
self.redis_client.rpush("Exports", key)
# TODO(rkn): If the worker fails after it calls setnx and before it
Expand Down Expand Up @@ -1446,12 +1446,8 @@ def _init(address_info=None,
# Use 1 local scheduler if num_local_schedulers is not provided. If
# existing local schedulers are provided, use that count as
# num_local_schedulers.
local_schedulers = address_info.get("local_scheduler_socket_names", [])
if num_local_schedulers is None:
if len(local_schedulers) > 0:
num_local_schedulers = len(local_schedulers)
else:
num_local_schedulers = 1
num_local_schedulers = 1
# Use 1 additional redis shard if num_redis_shards is not provided.
num_redis_shards = 1 if num_redis_shards is None else num_redis_shards

Expand Down Expand Up @@ -2013,21 +2009,20 @@ def connect(info,
"driver_id": worker.worker_id,
"start_time": time.time(),
"plasma_store_socket": info["store_socket_name"],
"local_scheduler_socket": info.get("local_scheduler_socket_name"),
"raylet_socket": info.get("raylet_socket_name")
}
driver_info["name"] = (main.__file__ if hasattr(main, "__file__") else
"INTERACTIVE MODE")
worker.redis_client.hmset(b"Drivers:" + worker.worker_id, driver_info)
if not worker.redis_client.exists("webui"):
if (not worker.redis_client.exists("webui")
and info["webui_url"] is not None):
worker.redis_client.hmset("webui", {"url": info["webui_url"]})
is_worker = False
elif mode == WORKER_MODE:
# Register the worker with Redis.
worker_dict = {
"node_ip_address": worker.node_ip_address,
"plasma_store_socket": info["store_socket_name"],
"local_scheduler_socket": info["local_scheduler_socket_name"]
}
if redirect_worker_output:
worker_dict["stdout_file"] = os.path.abspath(log_stdout_file.name)
Expand All @@ -2041,7 +2036,7 @@ def connect(info,
worker.plasma_client = thread_safe_client(
plasma.connect(info["store_socket_name"], "", 64))

local_scheduler_socket = info["raylet_socket_name"]
raylet_socket = info["raylet_socket_name"]

# If this is a driver, set the current task ID, the task driver ID, and set
# the task index to 0.
Expand Down Expand Up @@ -2100,8 +2095,7 @@ def connect(info,
worker.multithreading_warned = False

worker.local_scheduler_client = ray.raylet.LocalSchedulerClient(
local_scheduler_socket, worker.worker_id, is_worker,
worker.current_task_id)
raylet_socket, worker.worker_id, is_worker, worker.current_task_id)

# Start the import thread
import_thread.ImportThread(worker, mode).start()
Expand Down
6 changes: 0 additions & 6 deletions python/ray/workers/default_worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -40,11 +40,6 @@
required=False,
type=str,
help="the object store manager's name")
parser.add_argument(
"--local-scheduler-name",
required=False,
type=str,
help="the local scheduler's name")
parser.add_argument(
"--raylet-name", required=False, type=str, help="the raylet's name")
parser.add_argument(
Expand Down Expand Up @@ -76,7 +71,6 @@
"redis_password": args.redis_password,
"store_socket_name": args.object_store_name,
"manager_socket_name": args.object_store_manager_name,
"local_scheduler_socket_name": args.local_scheduler_name,
"raylet_socket_name": args.raylet_name
}

Expand Down
2 changes: 1 addition & 1 deletion python/setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ def find_version(*filepath):
"colorama",
"pytest",
"pyyaml",
"redis~=2.10.6",
"redis",
"setproctitle",
# The six module is required by pyarrow.
"six >= 1.0.0",
Expand Down
1 change: 0 additions & 1 deletion test/runtest.py
Original file line number Diff line number Diff line change
Expand Up @@ -2326,7 +2326,6 @@ def f():
assert len(worker_info) >= num_workers
for worker_id, info in worker_info.items():
assert "node_ip_address" in info
assert "local_scheduler_socket" in info
assert "plasma_store_socket" in info
assert "stderr_file" in info
assert "stdout_file" in info
Expand Down

0 comments on commit 3856533

Please sign in to comment.