mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Propagate backend error to worker (#4039)
This commit is contained in:
parent
4be3d0c5d3
commit
de17443dc2
21 changed files with 635 additions and 258 deletions
|
@ -19,22 +19,38 @@ import shlex
|
||||||
# These lines added to enable Sphinx to work without installing Ray.
|
# These lines added to enable Sphinx to work without installing Ray.
|
||||||
import mock
|
import mock
|
||||||
MOCK_MODULES = [
|
MOCK_MODULES = [
|
||||||
"gym", "gym.spaces", "scipy", "scipy.signal", "tensorflow",
|
"gym",
|
||||||
"tensorflow.contrib", "tensorflow.contrib.all_reduce",
|
"gym.spaces",
|
||||||
"tensorflow.contrib.all_reduce.python", "tensorflow.contrib.layers",
|
"ray._raylet",
|
||||||
"tensorflow.contrib.slim", "tensorflow.contrib.rnn", "tensorflow.core",
|
"ray.core.generated",
|
||||||
"tensorflow.core.util", "tensorflow.python", "tensorflow.python.client",
|
|
||||||
"tensorflow.python.util", "ray.core.generated",
|
|
||||||
"ray.core.generated.ActorCheckpointIdData",
|
"ray.core.generated.ActorCheckpointIdData",
|
||||||
"ray.core.generated.ClientTableData", "ray.core.generated.GcsTableEntry",
|
"ray.core.generated.ClientTableData",
|
||||||
"ray.core.generated.HeartbeatTableData",
|
"ray.core.generated.DriverTableData",
|
||||||
|
"ray.core.generated.ErrorTableData",
|
||||||
|
"ray.core.generated.ErrorType",
|
||||||
|
"ray.core.generated.GcsTableEntry",
|
||||||
"ray.core.generated.HeartbeatBatchTableData",
|
"ray.core.generated.HeartbeatBatchTableData",
|
||||||
"ray.core.generated.DriverTableData", "ray.core.generated.ErrorTableData",
|
"ray.core.generated.HeartbeatTableData",
|
||||||
"ray.core.generated.ProfileTableData",
|
"ray.core.generated.Language",
|
||||||
"ray.core.generated.ObjectTableData",
|
"ray.core.generated.ObjectTableData",
|
||||||
"ray.core.generated.ray.protocol.Task", "ray.core.generated.TablePrefix",
|
"ray.core.generated.ProfileTableData",
|
||||||
"ray.core.generated.TablePubsub", "ray.core.generated.Language",
|
"ray.core.generated.TablePrefix",
|
||||||
"ray._raylet"
|
"ray.core.generated.TablePubsub",
|
||||||
|
"ray.core.generated.ray.protocol.Task",
|
||||||
|
"scipy",
|
||||||
|
"scipy.signal",
|
||||||
|
"tensorflow",
|
||||||
|
"tensorflow.contrib",
|
||||||
|
"tensorflow.contrib.all_reduce",
|
||||||
|
"tensorflow.contrib.all_reduce.python",
|
||||||
|
"tensorflow.contrib.layers",
|
||||||
|
"tensorflow.contrib.rnn",
|
||||||
|
"tensorflow.contrib.slim",
|
||||||
|
"tensorflow.core",
|
||||||
|
"tensorflow.core.util",
|
||||||
|
"tensorflow.python",
|
||||||
|
"tensorflow.python.client",
|
||||||
|
"tensorflow.python.util",
|
||||||
]
|
]
|
||||||
for mod_name in MOCK_MODULES:
|
for mod_name in MOCK_MODULES:
|
||||||
sys.modules[mod_name] = mock.Mock()
|
sys.modules[mod_name] = mock.Mock()
|
||||||
|
|
|
@ -0,0 +1,16 @@
|
||||||
|
package org.ray.api.exception;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Indicates that the actor died unexpectedly before finishing a task.
|
||||||
|
*
|
||||||
|
* This exception could happen either because the actor process dies while executing a task, or
|
||||||
|
* because a task is submitted to a dead actor.
|
||||||
|
*/
|
||||||
|
public class RayActorException extends RayException {
|
||||||
|
|
||||||
|
public static final RayActorException INSTANCE = new RayActorException();
|
||||||
|
|
||||||
|
private RayActorException() {
|
||||||
|
super("The actor died unexpectedly before finishing this task.");
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,15 @@
|
||||||
|
package org.ray.api.exception;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Indicates that a task threw an exception during execution.
|
||||||
|
*
|
||||||
|
* If a task throws an exception during execution, a RayTaskException is stored in the object store
|
||||||
|
* as the task's output. Then when the object is retrieved from the object store, this exception
|
||||||
|
* will be thrown and propagate the error message.
|
||||||
|
*/
|
||||||
|
public class RayTaskException extends RayException {
|
||||||
|
|
||||||
|
public RayTaskException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,13 @@
|
||||||
|
package org.ray.api.exception;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Indicates that the worker died unexpectedly while executing a task.
|
||||||
|
*/
|
||||||
|
public class RayWorkerException extends RayException {
|
||||||
|
|
||||||
|
public static final RayWorkerException INSTANCE = new RayWorkerException();
|
||||||
|
|
||||||
|
private RayWorkerException() {
|
||||||
|
super("The worker died unexpectedly while executing this task.");
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,23 @@
|
||||||
|
package org.ray.api.exception;
|
||||||
|
|
||||||
|
import org.ray.api.id.UniqueId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Indicates that an object is lost (either evicted or explicitly deleted) and cannot be
|
||||||
|
* reconstructed.
|
||||||
|
*
|
||||||
|
* Note, this exception only happens for actor objects. If actor's current state is after object's
|
||||||
|
* creating task, the actor cannot re-run the task to reconstruct the object.
|
||||||
|
*/
|
||||||
|
public class UnreconstructableException extends RayException {
|
||||||
|
|
||||||
|
public final UniqueId objectId;
|
||||||
|
|
||||||
|
public UnreconstructableException(UniqueId objectId) {
|
||||||
|
super(String.format(
|
||||||
|
"Object %s is lost (either evicted or explicitly deleted) and cannot be reconstructed.",
|
||||||
|
objectId));
|
||||||
|
this.objectId = objectId;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -1,12 +1,15 @@
|
||||||
package org.ray.runtime;
|
package org.ray.runtime;
|
||||||
|
|
||||||
|
import static java.util.stream.Collectors.toList;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import org.apache.commons.lang3.tuple.Pair;
|
import java.util.stream.Collectors;
|
||||||
import org.ray.api.RayActor;
|
import org.ray.api.RayActor;
|
||||||
import org.ray.api.RayObject;
|
import org.ray.api.RayObject;
|
||||||
import org.ray.api.WaitResult;
|
import org.ray.api.WaitResult;
|
||||||
|
@ -21,7 +24,7 @@ import org.ray.runtime.config.RayConfig;
|
||||||
import org.ray.runtime.functionmanager.FunctionManager;
|
import org.ray.runtime.functionmanager.FunctionManager;
|
||||||
import org.ray.runtime.functionmanager.RayFunction;
|
import org.ray.runtime.functionmanager.RayFunction;
|
||||||
import org.ray.runtime.objectstore.ObjectStoreProxy;
|
import org.ray.runtime.objectstore.ObjectStoreProxy;
|
||||||
import org.ray.runtime.objectstore.ObjectStoreProxy.GetStatus;
|
import org.ray.runtime.objectstore.ObjectStoreProxy.GetResult;
|
||||||
import org.ray.runtime.raylet.RayletClient;
|
import org.ray.runtime.raylet.RayletClient;
|
||||||
import org.ray.runtime.task.ArgumentsBuilder;
|
import org.ray.runtime.task.ArgumentsBuilder;
|
||||||
import org.ray.runtime.task.TaskSpec;
|
import org.ray.runtime.task.TaskSpec;
|
||||||
|
@ -37,9 +40,22 @@ public abstract class AbstractRayRuntime implements RayRuntime {
|
||||||
|
|
||||||
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractRayRuntime.class);
|
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractRayRuntime.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default timeout of a get.
|
||||||
|
*/
|
||||||
private static final int GET_TIMEOUT_MS = 1000;
|
private static final int GET_TIMEOUT_MS = 1000;
|
||||||
|
/**
|
||||||
|
* Split objects in this batch size when fetching or reconstructing them.
|
||||||
|
*/
|
||||||
private static final int FETCH_BATCH_SIZE = 1000;
|
private static final int FETCH_BATCH_SIZE = 1000;
|
||||||
private static final int LIMITED_RETRY_COUNTER = 10;
|
/**
|
||||||
|
* Print a warning every this number of attempts.
|
||||||
|
*/
|
||||||
|
private static final int WARN_PER_NUM_ATTEMPTS = 50;
|
||||||
|
/**
|
||||||
|
* Max number of ids to print in the warning message.
|
||||||
|
*/
|
||||||
|
private static final int MAX_IDS_TO_PRINT_IN_WARNING = 20;
|
||||||
|
|
||||||
protected RayConfig rayConfig;
|
protected RayConfig rayConfig;
|
||||||
protected WorkerContext workerContext;
|
protected WorkerContext workerContext;
|
||||||
|
@ -75,7 +91,7 @@ public abstract class AbstractRayRuntime implements RayRuntime {
|
||||||
public <T> void put(UniqueId objectId, T obj) {
|
public <T> void put(UniqueId objectId, T obj) {
|
||||||
UniqueId taskId = workerContext.getCurrentTaskId();
|
UniqueId taskId = workerContext.getCurrentTaskId();
|
||||||
LOGGER.debug("Putting object {}, for task {} ", objectId, taskId);
|
LOGGER.debug("Putting object {}, for task {} ", objectId, taskId);
|
||||||
objectStoreProxy.put(objectId, obj, null);
|
objectStoreProxy.put(objectId, obj);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -90,7 +106,7 @@ public abstract class AbstractRayRuntime implements RayRuntime {
|
||||||
workerContext.getCurrentTaskId(), workerContext.nextPutIndex());
|
workerContext.getCurrentTaskId(), workerContext.nextPutIndex());
|
||||||
UniqueId taskId = workerContext.getCurrentTaskId();
|
UniqueId taskId = workerContext.getCurrentTaskId();
|
||||||
LOGGER.debug("Putting serialized object {}, for task {} ", objectId, taskId);
|
LOGGER.debug("Putting serialized object {}, for task {} ", objectId, taskId);
|
||||||
objectStoreProxy.putSerialized(objectId, obj, null);
|
objectStoreProxy.putSerialized(objectId, obj);
|
||||||
return new RayObjectImpl<>(objectId);
|
return new RayObjectImpl<>(objectId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -102,63 +118,68 @@ public abstract class AbstractRayRuntime implements RayRuntime {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <T> List<T> get(List<UniqueId> objectIds) {
|
public <T> List<T> get(List<UniqueId> objectIds) {
|
||||||
|
List<T> ret = new ArrayList<>(Collections.nCopies(objectIds.size(), null));
|
||||||
boolean wasBlocked = false;
|
boolean wasBlocked = false;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
int numObjectIds = objectIds.size();
|
// A map that stores the unready object ids and their original indexes.
|
||||||
|
Map<UniqueId, Integer> unready = new HashMap<>();
|
||||||
|
for (int i = 0; i < objectIds.size(); i++) {
|
||||||
|
unready.put(objectIds.get(i), i);
|
||||||
|
}
|
||||||
|
int numAttempts = 0;
|
||||||
|
|
||||||
// Do an initial fetch for remote objects.
|
// Repeat until we get all objects.
|
||||||
List<List<UniqueId>> fetchBatches = splitIntoBatches(objectIds);
|
while (!unready.isEmpty()) {
|
||||||
for (List<UniqueId> batch : fetchBatches) {
|
List<UniqueId> unreadyIds = new ArrayList<>(unready.keySet());
|
||||||
rayletClient.fetchOrReconstruct(batch, true, workerContext.getCurrentTaskId());
|
|
||||||
|
// For the initial fetch, we only fetch the objects, do not reconstruct them.
|
||||||
|
boolean fetchOnly = numAttempts == 0;
|
||||||
|
if (!fetchOnly) {
|
||||||
|
// If fetchOnly is false, this worker will be blocked.
|
||||||
|
wasBlocked = true;
|
||||||
|
}
|
||||||
|
// Call `fetchOrReconstruct` in batches.
|
||||||
|
for (List<UniqueId> batch : splitIntoBatches(unreadyIds)) {
|
||||||
|
rayletClient.fetchOrReconstruct(batch, fetchOnly, workerContext.getCurrentTaskId());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Get the objects. We initially try to get the objects immediately.
|
// Get the objects from the object store, and parse the result.
|
||||||
List<Pair<T, GetStatus>> ret = objectStoreProxy
|
List<GetResult<T>> getResults = objectStoreProxy.get(unreadyIds, GET_TIMEOUT_MS);
|
||||||
.get(objectIds, GET_TIMEOUT_MS, false);
|
for (int i = 0; i < getResults.size(); i++) {
|
||||||
assert ret.size() == numObjectIds;
|
GetResult<T> getResult = getResults.get(i);
|
||||||
|
if (getResult.exists) {
|
||||||
// Mapping the object IDs that we haven't gotten yet to their original index in objectIds.
|
if (getResult.exception != null) {
|
||||||
Map<UniqueId, Integer> unreadys = new HashMap<>();
|
// If the result is an exception, throw it.
|
||||||
for (int i = 0; i < numObjectIds; i++) {
|
throw getResult.exception;
|
||||||
if (ret.get(i).getRight() != GetStatus.SUCCESS) {
|
} else {
|
||||||
unreadys.put(objectIds.get(i), i);
|
// Set the result to the return list, and remove it from the unready map.
|
||||||
|
UniqueId id = unreadyIds.get(i);
|
||||||
|
ret.set(unready.get(id), getResult.object);
|
||||||
|
unready.remove(id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
wasBlocked = (unreadys.size() > 0);
|
|
||||||
|
|
||||||
// Try reconstructing any objects we haven't gotten yet. Try to get them
|
|
||||||
// until at least PlasmaLink.GET_TIMEOUT_MS milliseconds passes, then repeat.
|
|
||||||
int retryCounter = 0;
|
|
||||||
while (unreadys.size() > 0) {
|
|
||||||
retryCounter++;
|
|
||||||
List<UniqueId> unreadyList = new ArrayList<>(unreadys.keySet());
|
|
||||||
List<List<UniqueId>> reconstructBatches = splitIntoBatches(unreadyList);
|
|
||||||
|
|
||||||
for (List<UniqueId> batch : reconstructBatches) {
|
|
||||||
rayletClient.fetchOrReconstruct(batch, false, workerContext.getCurrentTaskId());
|
|
||||||
}
|
|
||||||
|
|
||||||
List<Pair<T, GetStatus>> results = objectStoreProxy
|
|
||||||
.get(unreadyList, GET_TIMEOUT_MS, false);
|
|
||||||
|
|
||||||
// Remove any entries for objects we received during this iteration so we
|
|
||||||
// don't retrieve the same object twice.
|
|
||||||
for (int i = 0; i < results.size(); i++) {
|
|
||||||
Pair<T, GetStatus> value = results.get(i);
|
|
||||||
if (value.getRight() == GetStatus.SUCCESS) {
|
|
||||||
UniqueId id = unreadyList.get(i);
|
|
||||||
ret.set(unreadys.get(id), value);
|
|
||||||
unreadys.remove(id);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (retryCounter % LIMITED_RETRY_COUNTER == 0) {
|
numAttempts += 1;
|
||||||
LOGGER.warn("Attempted {} times to reconstruct objects {}, "
|
if (LOGGER.isWarnEnabled() && numAttempts % WARN_PER_NUM_ATTEMPTS == 0) {
|
||||||
+ "but haven't received response. If this message continues to print,"
|
// Print a warning if we've attempted too many times, but some objects are still
|
||||||
+ " it may indicate that the task is hanging, or someting wrong "
|
// unavailable.
|
||||||
+ "happened in raylet backend.",
|
List<UniqueId> idsToPrint = new ArrayList<>(unready.keySet());
|
||||||
retryCounter, unreadys.keySet());
|
if (idsToPrint.size() > MAX_IDS_TO_PRINT_IN_WARNING) {
|
||||||
|
idsToPrint = idsToPrint.subList(0, MAX_IDS_TO_PRINT_IN_WARNING);
|
||||||
|
}
|
||||||
|
String ids = idsToPrint.stream().map(UniqueId::toString)
|
||||||
|
.collect(Collectors.joining(", "));
|
||||||
|
if (idsToPrint.size() < unready.size()) {
|
||||||
|
ids += ", etc";
|
||||||
|
}
|
||||||
|
String msg = String.format("Attempted %d times to reconstruct objects,"
|
||||||
|
+ " but some objects are still unavailable. If this message continues to print,"
|
||||||
|
+ " it may indicate that object's creating task is hanging, or something wrong"
|
||||||
|
+ " happened in raylet backend. %d object(s) pending: %s.", numAttempts,
|
||||||
|
unreadyIds.size(), ids);
|
||||||
|
LOGGER.warn(msg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -167,19 +188,10 @@ public abstract class AbstractRayRuntime implements RayRuntime {
|
||||||
workerContext.getCurrentTaskId());
|
workerContext.getCurrentTaskId());
|
||||||
}
|
}
|
||||||
|
|
||||||
List<T> finalRet = new ArrayList<>();
|
return ret;
|
||||||
|
|
||||||
for (Pair<T, GetStatus> value : ret) {
|
|
||||||
finalRet.add(value.getLeft());
|
|
||||||
}
|
|
||||||
|
|
||||||
return finalRet;
|
|
||||||
} catch (RayException e) {
|
|
||||||
LOGGER.error("Failed to get objects for task {}.", workerContext.getCurrentTaskId(), e);
|
|
||||||
throw e;
|
|
||||||
} finally {
|
} finally {
|
||||||
// If there were objects that we weren't able to get locally, let the local
|
// If there were objects that we weren't able to get locally, let the raylet backend
|
||||||
// scheduler know that we're now unblocked.
|
// know that we're now unblocked.
|
||||||
if (wasBlocked) {
|
if (wasBlocked) {
|
||||||
rayletClient.notifyUnblocked(workerContext.getCurrentTaskId());
|
rayletClient.notifyUnblocked(workerContext.getCurrentTaskId());
|
||||||
}
|
}
|
||||||
|
@ -252,6 +264,7 @@ public abstract class AbstractRayRuntime implements RayRuntime {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create the task specification.
|
* Create the task specification.
|
||||||
|
*
|
||||||
* @param func The target remote function.
|
* @param func The target remote function.
|
||||||
* @param actor The actor handle. If the task is not an actor task, actor id must be NIL.
|
* @param actor The actor handle. If the task is not an actor task, actor id must be NIL.
|
||||||
* @param args The arguments for the remote function.
|
* @param args The arguments for the remote function.
|
||||||
|
@ -323,6 +336,10 @@ public abstract class AbstractRayRuntime implements RayRuntime {
|
||||||
return rayletClient;
|
return rayletClient;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ObjectStoreProxy getObjectStoreProxy() {
|
||||||
|
return objectStoreProxy;
|
||||||
|
}
|
||||||
|
|
||||||
public FunctionManager getFunctionManager() {
|
public FunctionManager getFunctionManager() {
|
||||||
return functionManager;
|
return functionManager;
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,7 +6,7 @@ import java.util.List;
|
||||||
import org.ray.api.Checkpointable;
|
import org.ray.api.Checkpointable;
|
||||||
import org.ray.api.Checkpointable.Checkpoint;
|
import org.ray.api.Checkpointable.Checkpoint;
|
||||||
import org.ray.api.Checkpointable.CheckpointContext;
|
import org.ray.api.Checkpointable.CheckpointContext;
|
||||||
import org.ray.api.exception.RayException;
|
import org.ray.api.exception.RayTaskException;
|
||||||
import org.ray.api.id.UniqueId;
|
import org.ray.api.id.UniqueId;
|
||||||
import org.ray.runtime.config.RunMode;
|
import org.ray.runtime.config.RunMode;
|
||||||
import org.ray.runtime.functionmanager.RayFunction;
|
import org.ray.runtime.functionmanager.RayFunction;
|
||||||
|
@ -118,7 +118,7 @@ public class Worker {
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOGGER.error("Error executing task " + spec, e);
|
LOGGER.error("Error executing task " + spec, e);
|
||||||
if (!spec.isActorCreationTask()) {
|
if (!spec.isActorCreationTask()) {
|
||||||
runtime.put(returnId, new RayException("Error executing task " + spec, e));
|
runtime.put(returnId, new RayTaskException("Error executing task " + spec, e));
|
||||||
} else {
|
} else {
|
||||||
actorCreationException = e;
|
actorCreationException = e;
|
||||||
currentActorId = returnId;
|
currentActorId = returnId;
|
||||||
|
|
|
@ -1,30 +1,42 @@
|
||||||
package org.ray.runtime.objectstore;
|
package org.ray.runtime.objectstore;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import org.apache.arrow.plasma.ObjectStoreLink;
|
import org.apache.arrow.plasma.ObjectStoreLink;
|
||||||
|
import org.apache.arrow.plasma.ObjectStoreLink.ObjectStoreData;
|
||||||
import org.apache.arrow.plasma.PlasmaClient;
|
import org.apache.arrow.plasma.PlasmaClient;
|
||||||
import org.apache.arrow.plasma.exceptions.DuplicateObjectException;
|
import org.apache.arrow.plasma.exceptions.DuplicateObjectException;
|
||||||
import org.apache.commons.lang3.tuple.Pair;
|
import org.apache.commons.lang3.tuple.Pair;
|
||||||
|
import org.ray.api.exception.RayActorException;
|
||||||
import org.ray.api.exception.RayException;
|
import org.ray.api.exception.RayException;
|
||||||
|
import org.ray.api.exception.RayTaskException;
|
||||||
|
import org.ray.api.exception.RayWorkerException;
|
||||||
|
import org.ray.api.exception.UnreconstructableException;
|
||||||
import org.ray.api.id.UniqueId;
|
import org.ray.api.id.UniqueId;
|
||||||
import org.ray.runtime.AbstractRayRuntime;
|
import org.ray.runtime.AbstractRayRuntime;
|
||||||
import org.ray.runtime.RayDevRuntime;
|
import org.ray.runtime.RayDevRuntime;
|
||||||
import org.ray.runtime.config.RunMode;
|
import org.ray.runtime.config.RunMode;
|
||||||
|
import org.ray.runtime.generated.ErrorType;
|
||||||
import org.ray.runtime.util.Serializer;
|
import org.ray.runtime.util.Serializer;
|
||||||
import org.ray.runtime.util.UniqueIdUtil;
|
import org.ray.runtime.util.UniqueIdUtil;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Object store proxy, which handles serialization and deserialization, and utilize a {@code
|
* A class that is used to put/get objects to/from the object store.
|
||||||
* org.ray.spi.ObjectStoreLink} to actually store data.
|
|
||||||
*/
|
*/
|
||||||
public class ObjectStoreProxy {
|
public class ObjectStoreProxy {
|
||||||
|
|
||||||
private static final Logger LOGGER = LoggerFactory.getLogger(ObjectStoreProxy.class);
|
private static final Logger LOGGER = LoggerFactory.getLogger(ObjectStoreProxy.class);
|
||||||
|
|
||||||
private static final int GET_TIMEOUT_MS = 1000;
|
private static final byte[] WORKER_EXCEPTION_META = String.valueOf(ErrorType.WORKER_DIED)
|
||||||
|
.getBytes();
|
||||||
|
private static final byte[] ACTOR_EXCEPTION_META = String.valueOf(ErrorType.ACTOR_DIED)
|
||||||
|
.getBytes();
|
||||||
|
private static final byte[] UNRECONSTRUCTABLE_EXCEPTION_META = String
|
||||||
|
.valueOf(ErrorType.OBJECT_UNRECONSTRUCTABLE).getBytes();
|
||||||
|
|
||||||
private final AbstractRayRuntime runtime;
|
private final AbstractRayRuntime runtime;
|
||||||
|
|
||||||
|
@ -41,68 +53,134 @@ public class ObjectStoreProxy {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public <T> Pair<T, GetStatus> get(UniqueId objectId, boolean isMetadata)
|
/**
|
||||||
throws RayException {
|
* Get an object from the object store.
|
||||||
return get(objectId, GET_TIMEOUT_MS, isMetadata);
|
*
|
||||||
|
* @param id Id of the object.
|
||||||
|
* @param timeoutMs Timeout in milliseconds.
|
||||||
|
* @param <T> Type of the object.
|
||||||
|
* @return The GetResult object.
|
||||||
|
*/
|
||||||
|
public <T> GetResult<T> get(UniqueId id, int timeoutMs) {
|
||||||
|
List<GetResult<T>> list = get(ImmutableList.of(id), timeoutMs);
|
||||||
|
return list.get(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
public <T> Pair<T, GetStatus> get(UniqueId id, int timeoutMs, boolean isMetadata)
|
/**
|
||||||
throws RayException {
|
* Get a list of objects from the object store.
|
||||||
byte[] obj = objectStore.get().get(id.getBytes(), timeoutMs, isMetadata);
|
*
|
||||||
if (obj != null) {
|
* @param ids List of the object ids.
|
||||||
T t = Serializer.decode(obj, runtime.getWorkerContext().getCurrentClassLoader());
|
* @param timeoutMs Timeout in milliseconds.
|
||||||
objectStore.get().release(id.getBytes());
|
* @param <T> Type of these objects.
|
||||||
if (t instanceof RayException) {
|
* @return A list of GetResult objects.
|
||||||
throw (RayException) t;
|
*/
|
||||||
}
|
public <T> List<GetResult<T>> get(List<UniqueId> ids, int timeoutMs) {
|
||||||
return Pair.of(t, GetStatus.SUCCESS);
|
byte[][] binaryIds = UniqueIdUtil.getIdBytes(ids);
|
||||||
|
List<ObjectStoreData> dataAndMetaList = objectStore.get().get(binaryIds, timeoutMs);
|
||||||
|
|
||||||
|
List<GetResult<T>> results = new ArrayList<>();
|
||||||
|
for (int i = 0; i < dataAndMetaList.size(); i++) {
|
||||||
|
// TODO(hchen): Plasma API returns data and metadata in wrong order, this should be fixed
|
||||||
|
// from the arrow side first.
|
||||||
|
byte[] meta = dataAndMetaList.get(i).data;
|
||||||
|
byte[] data = dataAndMetaList.get(i).metadata;
|
||||||
|
|
||||||
|
GetResult<T> result;
|
||||||
|
if (meta != null) {
|
||||||
|
// If meta is not null, deserialize the exception.
|
||||||
|
RayException exception = deserializeRayExceptionFromMeta(meta, ids.get(i));
|
||||||
|
result = new GetResult<>(true, null, exception);
|
||||||
|
} else if (data != null) {
|
||||||
|
// If data is not null, deserialize the Java object.
|
||||||
|
Object object = Serializer.decode(data, runtime.getWorkerContext().getCurrentClassLoader());
|
||||||
|
if (object instanceof RayException) {
|
||||||
|
// If the object is a `RayException`, it means that an error occurred during task
|
||||||
|
// execution.
|
||||||
|
result = new GetResult<>(true, null, (RayException) object);
|
||||||
} else {
|
} else {
|
||||||
return Pair.of(null, GetStatus.FAILED);
|
// Otherwise, the object is valid.
|
||||||
|
result = new GetResult<>(true, (T) object, null);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
public <T> List<Pair<T, GetStatus>> get(List<UniqueId> objectIds, boolean isMetadata)
|
|
||||||
throws RayException {
|
|
||||||
return get(objectIds, GET_TIMEOUT_MS, isMetadata);
|
|
||||||
}
|
|
||||||
|
|
||||||
public <T> List<Pair<T, GetStatus>> get(List<UniqueId> ids, int timeoutMs, boolean isMetadata)
|
|
||||||
throws RayException {
|
|
||||||
List<byte[]> objs = objectStore.get().get(UniqueIdUtil.getIdBytes(ids), timeoutMs, isMetadata);
|
|
||||||
List<Pair<T, GetStatus>> ret = new ArrayList<>();
|
|
||||||
for (int i = 0; i < objs.size(); i++) {
|
|
||||||
byte[] obj = objs.get(i);
|
|
||||||
if (obj != null) {
|
|
||||||
T t = Serializer.decode(obj, runtime.getWorkerContext().getCurrentClassLoader());
|
|
||||||
objectStore.get().release(ids.get(i).getBytes());
|
|
||||||
if (t instanceof RayException) {
|
|
||||||
throw (RayException) t;
|
|
||||||
}
|
|
||||||
ret.add(Pair.of(t, GetStatus.SUCCESS));
|
|
||||||
} else {
|
} else {
|
||||||
ret.add(Pair.of(null, GetStatus.FAILED));
|
// If both meta and data are null, the object doesn't exist in object store.
|
||||||
}
|
result = new GetResult<>(false, null, null);
|
||||||
}
|
|
||||||
return ret;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void put(UniqueId id, Object obj, Object metadata) {
|
if (meta != null || data != null) {
|
||||||
|
// Release the object from object store..
|
||||||
|
objectStore.get().release(binaryIds[i]);
|
||||||
|
}
|
||||||
|
|
||||||
|
results.add(result);
|
||||||
|
}
|
||||||
|
return results;
|
||||||
|
}
|
||||||
|
|
||||||
|
private RayException deserializeRayExceptionFromMeta(byte[] meta, UniqueId objectId) {
|
||||||
|
if (Arrays.equals(meta, WORKER_EXCEPTION_META)) {
|
||||||
|
return RayWorkerException.INSTANCE;
|
||||||
|
} else if (Arrays.equals(meta, ACTOR_EXCEPTION_META)) {
|
||||||
|
return RayActorException.INSTANCE;
|
||||||
|
} else if (Arrays.equals(meta, UNRECONSTRUCTABLE_EXCEPTION_META)) {
|
||||||
|
return new UnreconstructableException(objectId);
|
||||||
|
}
|
||||||
|
throw new IllegalArgumentException("Unrecognized metadata " + Arrays.toString(meta));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Serialize and put an object to the object store.
|
||||||
|
*
|
||||||
|
* @param id Id of the object.
|
||||||
|
* @param object The object to put.
|
||||||
|
*/
|
||||||
|
public void put(UniqueId id, Object object) {
|
||||||
try {
|
try {
|
||||||
objectStore.get().put(id.getBytes(), Serializer.encode(obj), Serializer.encode(metadata));
|
objectStore.get().put(id.getBytes(), Serializer.encode(object), null);
|
||||||
} catch (DuplicateObjectException e) {
|
} catch (DuplicateObjectException e) {
|
||||||
LOGGER.warn(e.getMessage());
|
LOGGER.warn(e.getMessage());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void putSerialized(UniqueId id, byte[] obj, byte[] metadata) {
|
/**
|
||||||
|
* Put an already serialized object to the object store.
|
||||||
|
*
|
||||||
|
* @param id Id of the object.
|
||||||
|
* @param serializedObject The serialized object to put.
|
||||||
|
*/
|
||||||
|
public void putSerialized(UniqueId id, byte[] serializedObject) {
|
||||||
try {
|
try {
|
||||||
objectStore.get().put(id.getBytes(), obj, metadata);
|
objectStore.get().put(id.getBytes(), serializedObject, null);
|
||||||
} catch (DuplicateObjectException e) {
|
} catch (DuplicateObjectException e) {
|
||||||
LOGGER.warn(e.getMessage());
|
LOGGER.warn(e.getMessage());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public enum GetStatus {
|
/**
|
||||||
SUCCESS, FAILED
|
* A class that represents the result of a get operation.
|
||||||
|
*/
|
||||||
|
public static class GetResult<T> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Whether this object exists in object store.
|
||||||
|
*/
|
||||||
|
public final boolean exists;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The Java object that was fetched and deserialized from the object store. Note, this field
|
||||||
|
* only makes sense when @code{exists == true && exception !=null}.
|
||||||
|
*/
|
||||||
|
public final T object;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If this field is not null, it represents the exception that occurred during object's creating
|
||||||
|
* task.
|
||||||
|
*/
|
||||||
|
public final RayException exception;
|
||||||
|
|
||||||
|
GetResult(boolean exists, T object, RayException exception) {
|
||||||
|
this.exists = exists;
|
||||||
|
this.object = object;
|
||||||
|
this.exception = exception;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -10,6 +10,7 @@ import org.ray.api.Checkpointable;
|
||||||
import org.ray.api.Ray;
|
import org.ray.api.Ray;
|
||||||
import org.ray.api.RayActor;
|
import org.ray.api.RayActor;
|
||||||
import org.ray.api.annotation.RayRemote;
|
import org.ray.api.annotation.RayRemote;
|
||||||
|
import org.ray.api.exception.RayActorException;
|
||||||
import org.ray.api.id.UniqueId;
|
import org.ray.api.id.UniqueId;
|
||||||
import org.ray.api.options.ActorCreationOptions;
|
import org.ray.api.options.ActorCreationOptions;
|
||||||
import org.testng.Assert;
|
import org.testng.Assert;
|
||||||
|
@ -60,11 +61,8 @@ public class ActorReconstructionTest extends BaseTest {
|
||||||
try {
|
try {
|
||||||
Ray.call(Counter::increase, actor).get();
|
Ray.call(Counter::increase, actor).get();
|
||||||
Assert.fail("The above task didn't fail.");
|
Assert.fail("The above task didn't fail.");
|
||||||
} catch (StringIndexOutOfBoundsException e) {
|
} catch (RayActorException e) {
|
||||||
// Raylet backend will put invalid data in task's result to indicate the task has failed.
|
// We should receive a RayActorException because the actor is dead.
|
||||||
// Thus, Java deserialization will fail and throw `StringIndexOutOfBoundsException`.
|
|
||||||
// TODO(hchen): we should use object's metadata to indicate task failure,
|
|
||||||
// instead of throwing this exception.
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,11 +1,16 @@
|
||||||
package org.ray.api.test;
|
package org.ray.api.test;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import org.ray.api.Ray;
|
import org.ray.api.Ray;
|
||||||
import org.ray.api.RayActor;
|
import org.ray.api.RayActor;
|
||||||
import org.ray.api.RayObject;
|
import org.ray.api.RayObject;
|
||||||
import org.ray.api.annotation.RayRemote;
|
import org.ray.api.annotation.RayRemote;
|
||||||
|
import org.ray.api.exception.UnreconstructableException;
|
||||||
import org.ray.api.id.UniqueId;
|
import org.ray.api.id.UniqueId;
|
||||||
|
import org.ray.runtime.AbstractRayRuntime;
|
||||||
import org.ray.runtime.RayActorImpl;
|
import org.ray.runtime.RayActorImpl;
|
||||||
|
import org.ray.runtime.objectstore.ObjectStoreProxy.GetResult;
|
||||||
import org.testng.Assert;
|
import org.testng.Assert;
|
||||||
import org.testng.annotations.Test;
|
import org.testng.annotations.Test;
|
||||||
|
|
||||||
|
@ -83,4 +88,30 @@ public class ActorTest extends BaseTest {
|
||||||
Assert.assertEquals(Integer.valueOf(103), Ray.call(Counter::increase, counter2, 2).get());
|
Assert.assertEquals(Integer.valueOf(103), Ray.call(Counter::increase, counter2, 2).get());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnreconstructableActorObject() throws InterruptedException {
|
||||||
|
RayActor<Counter> counter = Ray.createActor(Counter::new, 100);
|
||||||
|
// Call an actor method.
|
||||||
|
RayObject value = Ray.call(Counter::getValue, counter);
|
||||||
|
Assert.assertEquals(100, value.get());
|
||||||
|
// Delete the object from the object store.
|
||||||
|
Ray.internal().free(ImmutableList.of(value.getId()), false);
|
||||||
|
// Wait until the object is deleted, because the above free operation is async.
|
||||||
|
while (true) {
|
||||||
|
GetResult<Integer> result = ((AbstractRayRuntime)
|
||||||
|
Ray.internal()).getObjectStoreProxy().get(value.getId(), 0);
|
||||||
|
if (!result.exists) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
TimeUnit.MILLISECONDS.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Try getting the object again, this should throw an UnreconstructableException.
|
||||||
|
value.get();
|
||||||
|
Assert.fail("This line should not be reachable.");
|
||||||
|
} catch (UnreconstructableException e) {
|
||||||
|
Assert.assertEquals(value.getId(), e.objectId);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -3,7 +3,9 @@ package org.ray.api.test;
|
||||||
import org.ray.api.Ray;
|
import org.ray.api.Ray;
|
||||||
import org.ray.api.RayActor;
|
import org.ray.api.RayActor;
|
||||||
import org.ray.api.RayObject;
|
import org.ray.api.RayObject;
|
||||||
import org.ray.api.exception.RayException;
|
import org.ray.api.exception.RayActorException;
|
||||||
|
import org.ray.api.exception.RayTaskException;
|
||||||
|
import org.ray.api.exception.RayWorkerException;
|
||||||
import org.testng.Assert;
|
import org.testng.Assert;
|
||||||
import org.testng.annotations.Test;
|
import org.testng.annotations.Test;
|
||||||
|
|
||||||
|
@ -15,6 +17,11 @@ public class FailureTest extends BaseTest {
|
||||||
throw new RuntimeException(EXCEPTION_MESSAGE);
|
throw new RuntimeException(EXCEPTION_MESSAGE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static int badFunc2() {
|
||||||
|
System.exit(-1);
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
public static class BadActor {
|
public static class BadActor {
|
||||||
|
|
||||||
public BadActor(boolean failOnCreation) {
|
public BadActor(boolean failOnCreation) {
|
||||||
|
@ -23,17 +30,21 @@ public class FailureTest extends BaseTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public int func() {
|
public int badMethod() {
|
||||||
throw new RuntimeException(EXCEPTION_MESSAGE);
|
throw new RuntimeException(EXCEPTION_MESSAGE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int badMethod2() {
|
||||||
|
System.exit(-1);
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void assertTaskFail(RayObject<?> rayObject) {
|
private static void assertTaskFailedWithRayTaskException(RayObject<?> rayObject) {
|
||||||
try {
|
try {
|
||||||
rayObject.get();
|
rayObject.get();
|
||||||
Assert.fail("Task didn't fail.");
|
Assert.fail("Task didn't fail.");
|
||||||
} catch (RayException e) {
|
} catch (RayTaskException e) {
|
||||||
e.printStackTrace();
|
|
||||||
Throwable rootCause = e.getCause();
|
Throwable rootCause = e.getCause();
|
||||||
while (rootCause.getCause() != null) {
|
while (rootCause.getCause() != null) {
|
||||||
rootCause = rootCause.getCause();
|
rootCause = rootCause.getCause();
|
||||||
|
@ -45,19 +56,49 @@ public class FailureTest extends BaseTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNormalTaskFailure() {
|
public void testNormalTaskFailure() {
|
||||||
assertTaskFail(Ray.call(FailureTest::badFunc));
|
assertTaskFailedWithRayTaskException(Ray.call(FailureTest::badFunc));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testActorCreationFailure() {
|
public void testActorCreationFailure() {
|
||||||
RayActor<BadActor> actor = Ray.createActor(BadActor::new, true);
|
RayActor<BadActor> actor = Ray.createActor(BadActor::new, true);
|
||||||
assertTaskFail(Ray.call(BadActor::func, actor));
|
assertTaskFailedWithRayTaskException(Ray.call(BadActor::badMethod, actor));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testActorTaskFailure() {
|
public void testActorTaskFailure() {
|
||||||
RayActor<BadActor> actor = Ray.createActor(BadActor::new, false);
|
RayActor<BadActor> actor = Ray.createActor(BadActor::new, false);
|
||||||
assertTaskFail(Ray.call(BadActor::func, actor));
|
assertTaskFailedWithRayTaskException(Ray.call(BadActor::badMethod, actor));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWorkerProcessDying() {
|
||||||
|
try {
|
||||||
|
Ray.call(FailureTest::badFunc2).get();
|
||||||
|
Assert.fail("This line shouldn't be reached.");
|
||||||
|
} catch (RayWorkerException e) {
|
||||||
|
// When the worker process dies while executing a task, we should receive an
|
||||||
|
// RayWorkerException.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testActorProcessDying() {
|
||||||
|
RayActor<BadActor> actor = Ray.createActor(BadActor::new, false);
|
||||||
|
try {
|
||||||
|
Ray.call(BadActor::badMethod2, actor).get();
|
||||||
|
Assert.fail("This line shouldn't be reached.");
|
||||||
|
} catch (RayActorException e) {
|
||||||
|
// When the actor process dies while executing a task, we should receive an
|
||||||
|
// RayActorException.
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
Ray.call(BadActor::badMethod, actor).get();
|
||||||
|
Assert.fail("This line shouldn't be reached.");
|
||||||
|
} catch (RayActorException e) {
|
||||||
|
// When a actor task is submitted to a dead actor, we should also receive an
|
||||||
|
// RayActorException.
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
105
python/ray/exceptions.py
Normal file
105
python/ray/exceptions.py
Normal file
|
@ -0,0 +1,105 @@
|
||||||
|
import os
|
||||||
|
|
||||||
|
import colorama
|
||||||
|
|
||||||
|
try:
|
||||||
|
import setproctitle
|
||||||
|
except ImportError:
|
||||||
|
setproctitle = None
|
||||||
|
|
||||||
|
|
||||||
|
class RayError(Exception):
|
||||||
|
"""Super class of all ray exception types."""
|
||||||
|
pass
|
||||||
|
|
||||||
|
|
||||||
|
class RayTaskError(RayError):
|
||||||
|
"""Indicates that a task threw an exception during execution.
|
||||||
|
|
||||||
|
If a task throws an exception during execution, a RayTaskError is stored in
|
||||||
|
the object store for each of the task's outputs. When an object is
|
||||||
|
retrieved from the object store, the Python method that retrieved it checks
|
||||||
|
to see if the object is a RayTaskError and if it is then an exception is
|
||||||
|
thrown propagating the error message.
|
||||||
|
|
||||||
|
Attributes:
|
||||||
|
function_name (str): The name of the function that failed and produced
|
||||||
|
the RayTaskError.
|
||||||
|
traceback_str (str): The traceback from the exception.
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __init__(self, function_name, traceback_str):
|
||||||
|
"""Initialize a RayTaskError."""
|
||||||
|
if setproctitle:
|
||||||
|
self.proctitle = setproctitle.getproctitle()
|
||||||
|
else:
|
||||||
|
self.proctitle = "ray_worker"
|
||||||
|
self.pid = os.getpid()
|
||||||
|
self.host = os.uname()[1]
|
||||||
|
self.function_name = function_name
|
||||||
|
self.traceback_str = traceback_str
|
||||||
|
assert traceback_str is not None
|
||||||
|
|
||||||
|
def __str__(self):
|
||||||
|
"""Format a RayTaskError as a string."""
|
||||||
|
lines = self.traceback_str.split("\n")
|
||||||
|
out = []
|
||||||
|
in_worker = False
|
||||||
|
for line in lines:
|
||||||
|
if line.startswith("Traceback "):
|
||||||
|
out.append("{}{}{} (pid={}, host={})".format(
|
||||||
|
colorama.Fore.CYAN, self.proctitle, colorama.Fore.RESET,
|
||||||
|
self.pid, self.host))
|
||||||
|
elif in_worker:
|
||||||
|
in_worker = False
|
||||||
|
elif "ray/worker.py" in line or "ray/function_manager.py" in line:
|
||||||
|
in_worker = True
|
||||||
|
else:
|
||||||
|
out.append(line)
|
||||||
|
return "\n".join(out)
|
||||||
|
|
||||||
|
|
||||||
|
class RayWorkerError(RayError):
|
||||||
|
"""Indicates that the worker died unexpectedly while executing a task."""
|
||||||
|
|
||||||
|
def __str__(self):
|
||||||
|
return "The worker died unexpectedly while executing this task."
|
||||||
|
|
||||||
|
|
||||||
|
class RayActorError(RayError):
|
||||||
|
"""Indicates that the actor died unexpectedly before finishing a task.
|
||||||
|
|
||||||
|
This exception could happen either because the actor process dies while
|
||||||
|
executing a task, or because a task is submitted to a dead actor.
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __str__(self):
|
||||||
|
return "The actor died unexpectedly before finishing this task."
|
||||||
|
|
||||||
|
|
||||||
|
class UnreconstructableError(RayError):
|
||||||
|
"""Indicates that an object is lost and cannot be reconstructed.
|
||||||
|
|
||||||
|
Note, this exception only happens for actor objects. If actor's current
|
||||||
|
state is after object's creating task, the actor cannot re-run the task to
|
||||||
|
reconstruct the object.
|
||||||
|
|
||||||
|
Attributes:
|
||||||
|
object_id: ID of the object.
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __init__(self, object_id):
|
||||||
|
self.object_id = object_id
|
||||||
|
|
||||||
|
def __str__(self):
|
||||||
|
return ("Object {} is lost (either evicted or explicitly deleted) and "
|
||||||
|
+ "cannot be reconstructed.").format(self.object_id.hex())
|
||||||
|
|
||||||
|
|
||||||
|
RAY_EXCEPTION_TYPES = [
|
||||||
|
RayError,
|
||||||
|
RayTaskError,
|
||||||
|
RayWorkerError,
|
||||||
|
RayActorError,
|
||||||
|
UnreconstructableError,
|
||||||
|
]
|
|
@ -4,7 +4,6 @@ from __future__ import print_function
|
||||||
|
|
||||||
from contextlib import contextmanager
|
from contextlib import contextmanager
|
||||||
import atexit
|
import atexit
|
||||||
import colorama
|
|
||||||
import faulthandler
|
import faulthandler
|
||||||
import hashlib
|
import hashlib
|
||||||
import inspect
|
import inspect
|
||||||
|
@ -28,18 +27,43 @@ import ray.experimental.state as state
|
||||||
import ray.gcs_utils
|
import ray.gcs_utils
|
||||||
import ray.memory_monitor as memory_monitor
|
import ray.memory_monitor as memory_monitor
|
||||||
import ray.node
|
import ray.node
|
||||||
|
import ray.parameter
|
||||||
|
import ray.ray_constants as ray_constants
|
||||||
import ray.remote_function
|
import ray.remote_function
|
||||||
import ray.serialization as serialization
|
import ray.serialization as serialization
|
||||||
import ray.services as services
|
import ray.services as services
|
||||||
import ray.signature
|
import ray.signature
|
||||||
import ray.ray_constants as ray_constants
|
|
||||||
|
from ray import (
|
||||||
|
ActorHandleID,
|
||||||
|
ActorID,
|
||||||
|
ClientID,
|
||||||
|
DriverID,
|
||||||
|
ObjectID,
|
||||||
|
TaskID,
|
||||||
|
)
|
||||||
from ray import import_thread
|
from ray import import_thread
|
||||||
from ray import ObjectID, DriverID, ActorID, ActorHandleID, ClientID, TaskID
|
|
||||||
from ray import profiling
|
from ray import profiling
|
||||||
from ray.function_manager import (FunctionActorManager, FunctionDescriptor)
|
from ray.core.generated.ErrorType import ErrorType
|
||||||
import ray.parameter
|
from ray.exceptions import (
|
||||||
from ray.utils import (check_oversized_pickle, is_cython, _random_string,
|
RayActorError,
|
||||||
thread_safe_client, setup_logger)
|
RayError,
|
||||||
|
RayTaskError,
|
||||||
|
RayWorkerError,
|
||||||
|
UnreconstructableError,
|
||||||
|
RAY_EXCEPTION_TYPES,
|
||||||
|
)
|
||||||
|
from ray.function_manager import (
|
||||||
|
FunctionActorManager,
|
||||||
|
FunctionDescriptor,
|
||||||
|
)
|
||||||
|
from ray.utils import (
|
||||||
|
_random_string,
|
||||||
|
check_oversized_pickle,
|
||||||
|
is_cython,
|
||||||
|
setup_logger,
|
||||||
|
thread_safe_client,
|
||||||
|
)
|
||||||
|
|
||||||
SCRIPT_MODE = 0
|
SCRIPT_MODE = 0
|
||||||
WORKER_MODE = 1
|
WORKER_MODE = 1
|
||||||
|
@ -68,55 +92,6 @@ except ImportError:
|
||||||
setproctitle = None
|
setproctitle = None
|
||||||
|
|
||||||
|
|
||||||
class RayTaskError(Exception):
|
|
||||||
"""An object used internally to represent a task that threw an exception.
|
|
||||||
|
|
||||||
If a task throws an exception during execution, a RayTaskError is stored in
|
|
||||||
the object store for each of the task's outputs. When an object is
|
|
||||||
retrieved from the object store, the Python method that retrieved it checks
|
|
||||||
to see if the object is a RayTaskError and if it is then an exception is
|
|
||||||
thrown propagating the error message.
|
|
||||||
|
|
||||||
Currently, we either use the exception attribute or the traceback attribute
|
|
||||||
but not both.
|
|
||||||
|
|
||||||
Attributes:
|
|
||||||
function_name (str): The name of the function that failed and produced
|
|
||||||
the RayTaskError.
|
|
||||||
traceback_str (str): The traceback from the exception.
|
|
||||||
"""
|
|
||||||
|
|
||||||
def __init__(self, function_name, traceback_str):
|
|
||||||
"""Initialize a RayTaskError."""
|
|
||||||
if setproctitle:
|
|
||||||
self.proctitle = setproctitle.getproctitle()
|
|
||||||
else:
|
|
||||||
self.proctitle = "ray_worker"
|
|
||||||
self.pid = os.getpid()
|
|
||||||
self.host = os.uname()[1]
|
|
||||||
self.function_name = function_name
|
|
||||||
self.traceback_str = traceback_str
|
|
||||||
assert traceback_str is not None
|
|
||||||
|
|
||||||
def __str__(self):
|
|
||||||
"""Format a RayTaskError as a string."""
|
|
||||||
lines = self.traceback_str.split("\n")
|
|
||||||
out = []
|
|
||||||
in_worker = False
|
|
||||||
for line in lines:
|
|
||||||
if line.startswith("Traceback "):
|
|
||||||
out.append("{}{}{} (pid={}, host={})".format(
|
|
||||||
colorama.Fore.CYAN, self.proctitle, colorama.Fore.RESET,
|
|
||||||
self.pid, self.host))
|
|
||||||
elif in_worker:
|
|
||||||
in_worker = False
|
|
||||||
elif "ray/worker.py" in line or "ray/function_manager.py" in line:
|
|
||||||
in_worker = True
|
|
||||||
else:
|
|
||||||
out.append(line)
|
|
||||||
return "\n".join(out)
|
|
||||||
|
|
||||||
|
|
||||||
class ActorCheckpointInfo(object):
|
class ActorCheckpointInfo(object):
|
||||||
"""Information used to maintain actor checkpoints."""
|
"""Information used to maintain actor checkpoints."""
|
||||||
|
|
||||||
|
@ -400,6 +375,8 @@ class Worker(object):
|
||||||
start_time = time.time()
|
start_time = time.time()
|
||||||
# Only send the warning once.
|
# Only send the warning once.
|
||||||
warning_sent = False
|
warning_sent = False
|
||||||
|
serialization_context = self.get_serialization_context(
|
||||||
|
self.task_driver_id)
|
||||||
while True:
|
while True:
|
||||||
try:
|
try:
|
||||||
# We divide very large get requests into smaller get requests
|
# We divide very large get requests into smaller get requests
|
||||||
|
@ -407,23 +384,23 @@ class Worker(object):
|
||||||
# long time, if the store is blocked, it can block the manager
|
# long time, if the store is blocked, it can block the manager
|
||||||
# as well as a consequence.
|
# as well as a consequence.
|
||||||
results = []
|
results = []
|
||||||
for i in range(0, len(object_ids),
|
batch_size = ray._config.worker_fetch_request_size()
|
||||||
ray._config.worker_get_request_size()):
|
for i in range(0, len(object_ids), batch_size):
|
||||||
results += self.plasma_client.get(
|
metadata_data_pairs = self.plasma_client.get_buffers(
|
||||||
object_ids[i:(
|
object_ids[i:i + batch_size],
|
||||||
i + ray._config.worker_get_request_size())],
|
|
||||||
timeout,
|
timeout,
|
||||||
self.get_serialization_context(self.task_driver_id))
|
with_meta=True,
|
||||||
|
)
|
||||||
|
for j in range(len(metadata_data_pairs)):
|
||||||
|
metadata, data = metadata_data_pairs[j]
|
||||||
|
results.append(
|
||||||
|
self._deserialize_object_from_arrow(
|
||||||
|
data,
|
||||||
|
metadata,
|
||||||
|
object_ids[i + j],
|
||||||
|
serialization_context,
|
||||||
|
))
|
||||||
return results
|
return results
|
||||||
except pyarrow.lib.ArrowInvalid:
|
|
||||||
# TODO(ekl): the local scheduler could include relevant
|
|
||||||
# metadata in the task kill case for a better error message
|
|
||||||
invalid_error = RayTaskError(
|
|
||||||
"<unknown>",
|
|
||||||
"Invalid return value: likely worker died or was killed "
|
|
||||||
"while executing the task; check previous logs or dmesg "
|
|
||||||
"for errors.")
|
|
||||||
return [invalid_error] * len(object_ids)
|
|
||||||
except pyarrow.DeserializationCallbackError:
|
except pyarrow.DeserializationCallbackError:
|
||||||
# Wait a little bit for the import thread to import the class.
|
# Wait a little bit for the import thread to import the class.
|
||||||
# If we currently have the worker lock, we need to release it
|
# If we currently have the worker lock, we need to release it
|
||||||
|
@ -448,6 +425,30 @@ class Worker(object):
|
||||||
driver_id=self.task_driver_id)
|
driver_id=self.task_driver_id)
|
||||||
warning_sent = True
|
warning_sent = True
|
||||||
|
|
||||||
|
def _deserialize_object_from_arrow(self, data, metadata, object_id,
|
||||||
|
serialization_context):
|
||||||
|
if metadata:
|
||||||
|
# If metadata is not empty, return an exception object based on
|
||||||
|
# the error type.
|
||||||
|
error_type = int(metadata)
|
||||||
|
if error_type == ErrorType.WORKER_DIED:
|
||||||
|
return RayWorkerError()
|
||||||
|
elif error_type == ErrorType.ACTOR_DIED:
|
||||||
|
return RayActorError()
|
||||||
|
elif error_type == ErrorType.OBJECT_UNRECONSTRUCTABLE:
|
||||||
|
return UnreconstructableError(ray.ObjectID(object_id.binary()))
|
||||||
|
else:
|
||||||
|
assert False, "Unrecognized error type " + str(error_type)
|
||||||
|
elif data:
|
||||||
|
# If data is not empty, deserialize the object.
|
||||||
|
# Note, the lock is needed because `serialization_context` isn't
|
||||||
|
# thread-safe.
|
||||||
|
with self.plasma_client.lock:
|
||||||
|
return pyarrow.deserialize(data, serialization_context)
|
||||||
|
else:
|
||||||
|
# Object isn't available in plasma.
|
||||||
|
return plasma.ObjectNotAvailable
|
||||||
|
|
||||||
def get_object(self, object_ids):
|
def get_object(self, object_ids):
|
||||||
"""Get the value or values in the object store associated with the IDs.
|
"""Get the value or values in the object store associated with the IDs.
|
||||||
|
|
||||||
|
@ -741,7 +742,7 @@ class Worker(object):
|
||||||
passed by value.
|
passed by value.
|
||||||
|
|
||||||
Raises:
|
Raises:
|
||||||
RayTaskError: This exception is raised if a task that
|
RayError: This exception is raised if a task that
|
||||||
created one of the arguments failed.
|
created one of the arguments failed.
|
||||||
"""
|
"""
|
||||||
arguments = []
|
arguments = []
|
||||||
|
@ -749,7 +750,7 @@ class Worker(object):
|
||||||
if isinstance(arg, ObjectID):
|
if isinstance(arg, ObjectID):
|
||||||
# get the object from the local object store
|
# get the object from the local object store
|
||||||
argument = self.get_object([arg])[0]
|
argument = self.get_object([arg])[0]
|
||||||
if isinstance(argument, RayTaskError):
|
if isinstance(argument, RayError):
|
||||||
raise argument
|
raise argument
|
||||||
else:
|
else:
|
||||||
# pass the argument by value
|
# pass the argument by value
|
||||||
|
@ -831,11 +832,6 @@ class Worker(object):
|
||||||
with profiling.profile("task:deserialize_arguments"):
|
with profiling.profile("task:deserialize_arguments"):
|
||||||
arguments = self._get_arguments_for_execution(
|
arguments = self._get_arguments_for_execution(
|
||||||
function_name, args)
|
function_name, args)
|
||||||
except RayTaskError as e:
|
|
||||||
self._handle_process_task_failure(
|
|
||||||
function_descriptor, return_object_ids, e,
|
|
||||||
ray.utils.format_error_message(traceback.format_exc()))
|
|
||||||
return
|
|
||||||
except Exception as e:
|
except Exception as e:
|
||||||
self._handle_process_task_failure(
|
self._handle_process_task_failure(
|
||||||
function_descriptor, return_object_ids, e,
|
function_descriptor, return_object_ids, e,
|
||||||
|
@ -1155,12 +1151,15 @@ def _initialize_serialization(driver_id, worker=global_worker):
|
||||||
|
|
||||||
worker.serialization_context_map[driver_id] = serialization_context
|
worker.serialization_context_map[driver_id] = serialization_context
|
||||||
|
|
||||||
|
# Register exception types.
|
||||||
|
for error_cls in RAY_EXCEPTION_TYPES:
|
||||||
register_custom_serializer(
|
register_custom_serializer(
|
||||||
RayTaskError,
|
error_cls,
|
||||||
use_dict=True,
|
use_dict=True,
|
||||||
local=True,
|
local=True,
|
||||||
driver_id=driver_id,
|
driver_id=driver_id,
|
||||||
class_id="ray.RayTaskError")
|
class_id=error_cls.__module__ + ". " + error_cls.__name__,
|
||||||
|
)
|
||||||
# Tell Ray to serialize lambdas with pickle.
|
# Tell Ray to serialize lambdas with pickle.
|
||||||
register_custom_serializer(
|
register_custom_serializer(
|
||||||
type(lambda: 0),
|
type(lambda: 0),
|
||||||
|
@ -2229,14 +2228,14 @@ def get(object_ids):
|
||||||
if isinstance(object_ids, list):
|
if isinstance(object_ids, list):
|
||||||
values = worker.get_object(object_ids)
|
values = worker.get_object(object_ids)
|
||||||
for i, value in enumerate(values):
|
for i, value in enumerate(values):
|
||||||
if isinstance(value, RayTaskError):
|
if isinstance(value, RayError):
|
||||||
last_task_error_raise_time = time.time()
|
last_task_error_raise_time = time.time()
|
||||||
raise value
|
raise value
|
||||||
return values
|
return values
|
||||||
else:
|
else:
|
||||||
value = worker.get_object([object_ids])[0]
|
value = worker.get_object([object_ids])[0]
|
||||||
if isinstance(value, RayTaskError):
|
if isinstance(value, RayError):
|
||||||
# If the result is a RayTaskError, then the task that created
|
# If the result is a RayError, then the task that created
|
||||||
# this object failed, and we should propagate the error message
|
# this object failed, and we should propagate the error message
|
||||||
# here.
|
# here.
|
||||||
last_task_error_raise_time = time.time()
|
last_task_error_raise_time = time.time()
|
||||||
|
|
|
@ -347,3 +347,23 @@ table ActorCheckpointIdData {
|
||||||
// A list of the timestamps for each of the above `checkpoint_ids`.
|
// A list of the timestamps for each of the above `checkpoint_ids`.
|
||||||
timestamps: [long];
|
timestamps: [long];
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// This enum type is used as object's metadata to indicate the object's creating
|
||||||
|
// task has failed because of a certain error.
|
||||||
|
// TODO(hchen): We may want to make these errors more specific. E.g., we may want
|
||||||
|
// to distinguish between intentional and expected actor failures, and between
|
||||||
|
// worker process failure and node failure.
|
||||||
|
enum ErrorType:int {
|
||||||
|
// Indicates that a task failed because the worker died unexpectedly while executing it.
|
||||||
|
WORKER_DIED = 1,
|
||||||
|
// Indicates that a task failed because the actor died unexpectedly before finishing it.
|
||||||
|
ACTOR_DIED = 2,
|
||||||
|
// Indicates that an object is lost and cannot be reconstructed.
|
||||||
|
// Note, this currently only happens to actor objects. When the actor's state is already
|
||||||
|
// after the object's creating task, the actor cannot re-run the task.
|
||||||
|
// TODO(hchen): we may want to reuse this error type for more cases. E.g.,
|
||||||
|
// 1) A object that was put by the driver.
|
||||||
|
// 2) The object's creating task is already cleaned up from GCS (this currently
|
||||||
|
// crashes raylet).
|
||||||
|
OBJECT_UNRECONSTRUCTABLE = 3,
|
||||||
|
}
|
||||||
|
|
|
@ -574,7 +574,7 @@ void NodeManager::HandleActorStateTransition(const ActorID &actor_id,
|
||||||
auto tasks_to_remove = local_queues_.GetTaskIdsForActor(actor_id);
|
auto tasks_to_remove = local_queues_.GetTaskIdsForActor(actor_id);
|
||||||
auto removed_tasks = local_queues_.RemoveTasks(tasks_to_remove);
|
auto removed_tasks = local_queues_.RemoveTasks(tasks_to_remove);
|
||||||
for (auto const &task : removed_tasks) {
|
for (auto const &task : removed_tasks) {
|
||||||
TreatTaskAsFailed(task);
|
TreatTaskAsFailed(task, ErrorType::ACTOR_DIED);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
RAY_CHECK(actor_registration.GetState() == ActorState::RECONSTRUCTING);
|
RAY_CHECK(actor_registration.GetState() == ActorState::RECONSTRUCTING);
|
||||||
|
@ -858,7 +858,7 @@ void NodeManager::ProcessDisconnectClientMessage(
|
||||||
// `HandleDisconnectedActor`.
|
// `HandleDisconnectedActor`.
|
||||||
if (actor_id.is_nil()) {
|
if (actor_id.is_nil()) {
|
||||||
const Task &task = local_queues_.RemoveTask(task_id);
|
const Task &task = local_queues_.RemoveTask(task_id);
|
||||||
TreatTaskAsFailed(task);
|
TreatTaskAsFailed(task, ErrorType::WORKER_DIED);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!intentional_disconnect) {
|
if (!intentional_disconnect) {
|
||||||
|
@ -1214,9 +1214,10 @@ bool NodeManager::CheckDependencyManagerInvariant() const {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
void NodeManager::TreatTaskAsFailed(const Task &task) {
|
void NodeManager::TreatTaskAsFailed(const Task &task, const ErrorType &error_type) {
|
||||||
const TaskSpecification &spec = task.GetTaskSpecification();
|
const TaskSpecification &spec = task.GetTaskSpecification();
|
||||||
RAY_LOG(DEBUG) << "Treating task " << spec.TaskId() << " as failed.";
|
RAY_LOG(DEBUG) << "Treating task " << spec.TaskId() << " as failed because of error "
|
||||||
|
<< EnumNameErrorType(error_type) << ".";
|
||||||
// If this was an actor creation task that tried to resume from a checkpoint,
|
// If this was an actor creation task that tried to resume from a checkpoint,
|
||||||
// then erase it here since the task did not finish.
|
// then erase it here since the task did not finish.
|
||||||
if (spec.IsActorCreationTask()) {
|
if (spec.IsActorCreationTask()) {
|
||||||
|
@ -1231,20 +1232,22 @@ void NodeManager::TreatTaskAsFailed(const Task &task) {
|
||||||
// information about the TaskSpecification implementation.
|
// information about the TaskSpecification implementation.
|
||||||
num_returns -= 1;
|
num_returns -= 1;
|
||||||
}
|
}
|
||||||
|
const std::string meta = std::to_string(static_cast<int>(error_type));
|
||||||
for (int64_t i = 0; i < num_returns; i++) {
|
for (int64_t i = 0; i < num_returns; i++) {
|
||||||
const ObjectID object_id = spec.ReturnId(i);
|
const auto object_id = spec.ReturnId(i).to_plasma_id();
|
||||||
|
arrow::Status status = store_client_.CreateAndSeal(object_id, "", meta);
|
||||||
std::shared_ptr<Buffer> data;
|
if (!status.ok() && !status.IsPlasmaObjectExists()) {
|
||||||
// TODO(ekl): this writes an invalid arrow object, which is sufficient to
|
// If we failed to save the error code, log a warning and push an error message
|
||||||
// signal that the worker failed, but it would be nice to return more
|
// to the driver.
|
||||||
// detailed failure metadata in the future.
|
std::ostringstream stream;
|
||||||
arrow::Status status =
|
stream << "An plasma error (" << status.ToString() << ") occurred while saving"
|
||||||
store_client_.Create(object_id.to_plasma_id(), 1, NULL, 0, &data);
|
<< " error code to object " << object_id << ". Anyone who's getting this"
|
||||||
if (!status.IsPlasmaObjectExists()) {
|
<< " object may hang forever.";
|
||||||
// TODO(rkn): We probably don't want this checks. E.g., if the object
|
std::string error_message = stream.str();
|
||||||
// store is full, we don't want to kill the raylet.
|
RAY_LOG(WARNING) << error_message;
|
||||||
RAY_ARROW_CHECK_OK(status);
|
RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver(
|
||||||
RAY_ARROW_CHECK_OK(store_client_.Seal(object_id.to_plasma_id()));
|
task.GetTaskSpecification().DriverId(), "task", error_message,
|
||||||
|
current_time_ms()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// A task failing is equivalent to assigning and finishing the task, so clean
|
// A task failing is equivalent to assigning and finishing the task, so clean
|
||||||
|
@ -1297,7 +1300,7 @@ void NodeManager::TreatTaskAsFailedIfLost(const Task &task) {
|
||||||
// The object does not exist on any nodes but has been created
|
// The object does not exist on any nodes but has been created
|
||||||
// before, so the object has been lost. Mark the task as failed to
|
// before, so the object has been lost. Mark the task as failed to
|
||||||
// prevent any tasks that depend on this object from hanging.
|
// prevent any tasks that depend on this object from hanging.
|
||||||
TreatTaskAsFailed(task);
|
TreatTaskAsFailed(task, ErrorType::OBJECT_UNRECONSTRUCTABLE);
|
||||||
*task_marked_as_failed = true;
|
*task_marked_as_failed = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1343,7 +1346,7 @@ void NodeManager::SubmitTask(const Task &task, const Lineage &uncommitted_lineag
|
||||||
if (actor_entry->second.GetState() == ActorState::DEAD) {
|
if (actor_entry->second.GetState() == ActorState::DEAD) {
|
||||||
// If this actor is dead, either because the actor process is dead
|
// If this actor is dead, either because the actor process is dead
|
||||||
// or because its residing node is dead, treat this task as failed.
|
// or because its residing node is dead, treat this task as failed.
|
||||||
TreatTaskAsFailed(task);
|
TreatTaskAsFailed(task, ErrorType::ACTOR_DIED);
|
||||||
} else {
|
} else {
|
||||||
// If this actor is alive, check whether this actor is local.
|
// If this actor is alive, check whether this actor is local.
|
||||||
auto node_manager_id = actor_entry->second.GetNodeManagerId();
|
auto node_manager_id = actor_entry->second.GetNodeManagerId();
|
||||||
|
|
|
@ -157,8 +157,9 @@ class NodeManager {
|
||||||
/// the local queue.
|
/// the local queue.
|
||||||
///
|
///
|
||||||
/// \param task The task to fail.
|
/// \param task The task to fail.
|
||||||
|
/// \param error_type The type of the error that caused this task to fail.
|
||||||
/// \return Void.
|
/// \return Void.
|
||||||
void TreatTaskAsFailed(const Task &task);
|
void TreatTaskAsFailed(const Task &task, const ErrorType &error_type);
|
||||||
/// This is similar to TreatTaskAsFailed, but it will only mark the task as
|
/// This is similar to TreatTaskAsFailed, but it will only mark the task as
|
||||||
/// failed if at least one of the task's return values is lost. A return
|
/// failed if at least one of the task's return values is lost. A return
|
||||||
/// value is lost if it has been created before, but no longer exists on any
|
/// value is lost if it has been created before, but no longer exists on any
|
||||||
|
|
|
@ -1405,7 +1405,7 @@ def test_exception_raised_when_actor_node_dies(head_node_cluster):
|
||||||
# Submit some new actor tasks.
|
# Submit some new actor tasks.
|
||||||
x_ids = [actor.inc.remote() for _ in range(5)]
|
x_ids = [actor.inc.remote() for _ in range(5)]
|
||||||
for x_id in x_ids:
|
for x_id in x_ids:
|
||||||
with pytest.raises(ray.worker.RayTaskError):
|
with pytest.raises(ray.exceptions.RayActorError):
|
||||||
# There is some small chance that ray.get will actually
|
# There is some small chance that ray.get will actually
|
||||||
# succeed (if the object is transferred before the raylet
|
# succeed (if the object is transferred before the raylet
|
||||||
# dies).
|
# dies).
|
||||||
|
@ -2128,7 +2128,7 @@ def test_actor_eviction(shutdown_only):
|
||||||
try:
|
try:
|
||||||
ray.get(obj)
|
ray.get(obj)
|
||||||
num_success += 1
|
num_success += 1
|
||||||
except ray.worker.RayTaskError:
|
except ray.exceptions.UnreconstructableError:
|
||||||
num_evicted += 1
|
num_evicted += 1
|
||||||
# Some objects should have been evicted, and some should still be in the
|
# Some objects should have been evicted, and some should still be in the
|
||||||
# object store.
|
# object store.
|
||||||
|
@ -2173,7 +2173,7 @@ def test_actor_reconstruction(ray_start_regular):
|
||||||
pid = ray.get(actor.get_pid.remote())
|
pid = ray.get(actor.get_pid.remote())
|
||||||
os.kill(pid, signal.SIGKILL)
|
os.kill(pid, signal.SIGKILL)
|
||||||
# The actor has exceeded max reconstructions, and this task should fail.
|
# The actor has exceeded max reconstructions, and this task should fail.
|
||||||
with pytest.raises(ray.worker.RayTaskError):
|
with pytest.raises(ray.exceptions.RayActorError):
|
||||||
ray.get(actor.increase.remote())
|
ray.get(actor.increase.remote())
|
||||||
|
|
||||||
# Create another actor.
|
# Create another actor.
|
||||||
|
@ -2181,7 +2181,7 @@ def test_actor_reconstruction(ray_start_regular):
|
||||||
# Intentionlly exit the actor
|
# Intentionlly exit the actor
|
||||||
actor.__ray_terminate__.remote()
|
actor.__ray_terminate__.remote()
|
||||||
# Check that the actor won't be reconstructed.
|
# Check that the actor won't be reconstructed.
|
||||||
with pytest.raises(ray.worker.RayTaskError):
|
with pytest.raises(ray.exceptions.RayActorError):
|
||||||
ray.get(actor.increase.remote())
|
ray.get(actor.increase.remote())
|
||||||
|
|
||||||
|
|
||||||
|
@ -2241,7 +2241,7 @@ def test_actor_reconstruction_on_node_failure(head_node_cluster):
|
||||||
object_store_socket = ray.get(actor.get_object_store_socket.remote())
|
object_store_socket = ray.get(actor.get_object_store_socket.remote())
|
||||||
kill_node(object_store_socket)
|
kill_node(object_store_socket)
|
||||||
# The actor has exceeded max reconstructions, and this task should fail.
|
# The actor has exceeded max reconstructions, and this task should fail.
|
||||||
with pytest.raises(ray.worker.RayTaskError):
|
with pytest.raises(ray.exceptions.RayActorError):
|
||||||
ray.get(actor.increase.remote())
|
ray.get(actor.increase.remote())
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -279,7 +279,7 @@ def test_worker_failed(ray_start_workers_separate_multinode):
|
||||||
for object_id in object_ids:
|
for object_id in object_ids:
|
||||||
try:
|
try:
|
||||||
ray.get(object_id)
|
ray.get(object_id)
|
||||||
except ray.worker.RayTaskError:
|
except (ray.exceptions.RayTaskError, ray.exceptions.RayWorkerError):
|
||||||
pass
|
pass
|
||||||
|
|
||||||
|
|
||||||
|
@ -424,7 +424,7 @@ def test_actor_creation_node_failure(ray_start_cluster):
|
||||||
for i, out in enumerate(children_out):
|
for i, out in enumerate(children_out):
|
||||||
try:
|
try:
|
||||||
ray.get(out)
|
ray.get(out)
|
||||||
except ray.worker.RayTaskError:
|
except ray.exceptions.RayActorError:
|
||||||
children[i] = Child.remote(death_probability)
|
children[i] = Child.remote(death_probability)
|
||||||
# Remove a node. Any actor creation tasks that were forwarded to this
|
# Remove a node. Any actor creation tasks that were forwarded to this
|
||||||
# node must be reconstructed.
|
# node must be reconstructed.
|
||||||
|
|
|
@ -319,7 +319,8 @@ def test_worker_dying(ray_start_regular):
|
||||||
def f():
|
def f():
|
||||||
eval("exit()")
|
eval("exit()")
|
||||||
|
|
||||||
f.remote()
|
with pytest.raises(ray.exceptions.RayWorkerError):
|
||||||
|
ray.get(f.remote())
|
||||||
|
|
||||||
wait_for_errors(ray_constants.WORKER_DIED_PUSH_ERROR, 1)
|
wait_for_errors(ray_constants.WORKER_DIED_PUSH_ERROR, 1)
|
||||||
|
|
||||||
|
@ -340,9 +341,9 @@ def test_actor_worker_dying(ray_start_regular):
|
||||||
|
|
||||||
a = Actor.remote()
|
a = Actor.remote()
|
||||||
[obj], _ = ray.wait([a.kill.remote()], timeout=5.0)
|
[obj], _ = ray.wait([a.kill.remote()], timeout=5.0)
|
||||||
with pytest.raises(Exception):
|
with pytest.raises(ray.exceptions.RayActorError):
|
||||||
ray.get(obj)
|
ray.get(obj)
|
||||||
with pytest.raises(Exception):
|
with pytest.raises(ray.exceptions.RayTaskError):
|
||||||
ray.get(consume.remote(obj))
|
ray.get(consume.remote(obj))
|
||||||
wait_for_errors(ray_constants.WORKER_DIED_PUSH_ERROR, 1)
|
wait_for_errors(ray_constants.WORKER_DIED_PUSH_ERROR, 1)
|
||||||
|
|
||||||
|
|
|
@ -2621,7 +2621,7 @@ def test_inline_objects(shutdown_only):
|
||||||
value = ray.get(inline_object)
|
value = ray.get(inline_object)
|
||||||
assert value == "inline"
|
assert value == "inline"
|
||||||
inlined += 1
|
inlined += 1
|
||||||
except ray.worker.RayTaskError:
|
except ray.exceptions.UnreconstructableError:
|
||||||
pass
|
pass
|
||||||
# Make sure some objects were inlined. Some of them may not get inlined
|
# Make sure some objects were inlined. Some of them may not get inlined
|
||||||
# because we evict the object soon after creating it.
|
# because we evict the object soon after creating it.
|
||||||
|
@ -2638,7 +2638,7 @@ def test_inline_objects(shutdown_only):
|
||||||
ray.worker.global_worker.plasma_client.delete([plasma_id])
|
ray.worker.global_worker.plasma_client.delete([plasma_id])
|
||||||
# Objects created by an actor that were evicted and larger than the
|
# Objects created by an actor that were evicted and larger than the
|
||||||
# maximum inline object size cannot be retrieved or reconstructed.
|
# maximum inline object size cannot be retrieved or reconstructed.
|
||||||
with pytest.raises(ray.worker.RayTaskError):
|
with pytest.raises(ray.exceptions.UnreconstructableError):
|
||||||
ray.get(non_inline_object) == 10000 * [1]
|
ray.get(non_inline_object) == 10000 * [1]
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -106,7 +106,7 @@ def test_task_crash(ray_start):
|
||||||
try:
|
try:
|
||||||
ray.get(object_id)
|
ray.get(object_id)
|
||||||
except Exception as e:
|
except Exception as e:
|
||||||
assert type(e) == ray.worker.RayTaskError
|
assert type(e) == ray.exceptions.RayTaskError
|
||||||
finally:
|
finally:
|
||||||
result_list = signal.receive([object_id], timeout=5)
|
result_list = signal.receive([object_id], timeout=5)
|
||||||
assert len(result_list) == 1
|
assert len(result_list) == 1
|
||||||
|
@ -142,7 +142,7 @@ def test_actor_crash(ray_start):
|
||||||
try:
|
try:
|
||||||
ray.get(a.crash.remote())
|
ray.get(a.crash.remote())
|
||||||
except Exception as e:
|
except Exception as e:
|
||||||
assert type(e) == ray.worker.RayTaskError
|
assert type(e) == ray.exceptions.RayTaskError
|
||||||
finally:
|
finally:
|
||||||
result_list = signal.receive([a], timeout=5)
|
result_list = signal.receive([a], timeout=5)
|
||||||
assert len(result_list) == 1
|
assert len(result_list) == 1
|
||||||
|
@ -184,7 +184,7 @@ def test_actor_crash_init2(ray_start):
|
||||||
try:
|
try:
|
||||||
ray.get(a.method.remote())
|
ray.get(a.method.remote())
|
||||||
except Exception as e:
|
except Exception as e:
|
||||||
assert type(e) == ray.worker.RayTaskError
|
assert type(e) == ray.exceptions.RayTaskError
|
||||||
finally:
|
finally:
|
||||||
result_list = receive_all_signals([a], timeout=5)
|
result_list = receive_all_signals([a], timeout=5)
|
||||||
assert len(result_list) == 2
|
assert len(result_list) == 2
|
||||||
|
|
Loading…
Add table
Reference in a new issue