Skip to content

Commit

Permalink
[ID Refactor] Rename DriverID to JobID (ray-project#5004)
Browse files Browse the repository at this point in the history
* WIP

WIP

WIP

Rename Driver -> Job

Fix complition

Fix

Rename in Java

In py

WIP

Fix

WIP

Fix

Fix test

Fix

Fix C++ linting

Fix

* Update java/runtime/src/main/java/org/ray/runtime/config/RayConfig.java

Co-Authored-By: Stephanie Wang <swang@cs.berkeley.edu>

* Update src/ray/core_worker/core_worker.cc

Co-Authored-By: Stephanie Wang <swang@cs.berkeley.edu>

* Address comments

* Fix

* Fix CI

* Fix cpp linting

* Fix py lint

* FIx

* Address comments and fix

* Address comments

* Address

* Fix import_threading
  • Loading branch information
jovany-wang authored Jun 27, 2019
1 parent d9768c1 commit 62e4b59
Show file tree
Hide file tree
Showing 79 changed files with 961 additions and 974 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -9,12 +9,9 @@
public interface RuntimeContext {

/**
* Get the current Driver ID.
*
* If called in a driver, this returns the driver ID. If called in a worker, this returns the ID
* of the associated driver.
* Get the current Job ID.
*/
UniqueId getCurrentDriverId();
UniqueId getCurrentJobId();

/**
* Get the current actor ID.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,10 +74,10 @@ public abstract class AbstractRayRuntime implements RayRuntime {

public AbstractRayRuntime(RayConfig rayConfig) {
this.rayConfig = rayConfig;
functionManager = new FunctionManager(rayConfig.driverResourcePath);
functionManager = new FunctionManager(rayConfig.jobResourcePath);
worker = new Worker(this);
workerContext = new WorkerContext(rayConfig.workerMode,
rayConfig.driverId, rayConfig.runMode);
rayConfig.jobId, rayConfig.runMode);
runtimeContext = new RuntimeContextImpl(this);
}

Expand Down Expand Up @@ -346,7 +346,7 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes
boolean isActorCreationTask, BaseTaskOptions taskOptions) {
Preconditions.checkArgument((func == null) != (pyFunctionDescriptor == null));

TaskId taskId = rayletClient.generateTaskId(workerContext.getCurrentDriverId(),
TaskId taskId = rayletClient.generateTaskId(workerContext.getCurrentJobId(),
workerContext.getCurrentTaskId(), workerContext.nextTaskIndex());
int numReturns = actor.getId().isNil() ? 1 : 2;
ObjectId[] returnIds = IdUtil.genReturnIds(taskId, numReturns);
Expand Down Expand Up @@ -377,15 +377,15 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes
FunctionDescriptor functionDescriptor;
if (func != null) {
language = TaskLanguage.JAVA;
functionDescriptor = functionManager.getFunction(workerContext.getCurrentDriverId(), func)
functionDescriptor = functionManager.getFunction(workerContext.getCurrentJobId(), func)
.getFunctionDescriptor();
} else {
language = TaskLanguage.PYTHON;
functionDescriptor = pyFunctionDescriptor;
}

return new TaskSpec(
workerContext.getCurrentDriverId(),
workerContext.getCurrentJobId(),
taskId,
workerContext.getCurrentTaskId(),
-1,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ public void start() {
rayConfig.rayletSocketName,
workerContext.getCurrentWorkerId(),
rayConfig.workerMode == WorkerMode.WORKER,
workerContext.getCurrentDriverId()
workerContext.getCurrentJobId()
);

// register
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@ public RuntimeContextImpl(AbstractRayRuntime runtime) {
}

@Override
public UniqueId getCurrentDriverId() {
return runtime.getWorkerContext().getCurrentDriverId();
public UniqueId getCurrentJobId() {
return runtime.getWorkerContext().getCurrentJobId();
}

@Override
Expand Down
2 changes: 1 addition & 1 deletion java/runtime/src/main/java/org/ray/runtime/Worker.java
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ public void execute(TaskSpec spec) {
try {
// Get method
RayFunction rayFunction = runtime.getFunctionManager()
.getFunction(spec.driverId, spec.getJavaFunctionDescriptor());
.getFunction(spec.jobId, spec.getJavaFunctionDescriptor());
// Set context
runtime.getWorkerContext().setCurrentTask(spec, rayFunction.classLoader);
Thread.currentThread().setContextClassLoader(rayFunction.classLoader);
Expand Down
23 changes: 12 additions & 11 deletions java/runtime/src/main/java/org/ray/runtime/WorkerContext.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ public class WorkerContext {

private ThreadLocal<TaskSpec> currentTask;

private UniqueId currentDriverId;
private UniqueId currentJobId;

private ClassLoader currentClassLoader;

Expand All @@ -43,7 +43,7 @@ public class WorkerContext {
*/
private RunMode runMode;

public WorkerContext(WorkerMode workerMode, UniqueId driverId, RunMode runMode) {
public WorkerContext(WorkerMode workerMode, UniqueId jobId, RunMode runMode) {
mainThreadId = Thread.currentThread().getId();
taskIndex = ThreadLocal.withInitial(() -> 0);
putIndex = ThreadLocal.withInitial(() -> 0);
Expand All @@ -52,13 +52,15 @@ public WorkerContext(WorkerMode workerMode, UniqueId driverId, RunMode runMode)
currentTask = ThreadLocal.withInitial(() -> null);
currentClassLoader = null;
if (workerMode == WorkerMode.DRIVER) {
workerId = driverId;
// TODO(qwang): Assign the driver id to worker id
// once we treat driver id as a special worker id.
workerId = jobId;
currentTaskId.set(TaskId.randomId());
currentDriverId = driverId;
currentJobId = jobId;
} else {
workerId = UniqueId.randomId();
this.currentTaskId.set(TaskId.NIL);
this.currentDriverId = UniqueId.NIL;
this.currentJobId = UniqueId.NIL;
}
}

Expand All @@ -84,7 +86,7 @@ public void setCurrentTask(TaskSpec task, ClassLoader classLoader) {

Preconditions.checkNotNull(task);
this.currentTaskId.set(task.taskId);
this.currentDriverId = task.driverId;
this.currentJobId = task.jobId;
taskIndex.set(0);
putIndex.set(0);
this.currentTask.set(task);
Expand Down Expand Up @@ -115,15 +117,14 @@ public UniqueId getCurrentWorkerId() {
}

/**
* @return If this worker is a driver, this method returns the driver ID; Otherwise, it returns
* the driver ID of the current running task.
* The ID of the current job.
*/
public UniqueId getCurrentDriverId() {
return currentDriverId;
public UniqueId getCurrentJobId() {
return currentJobId;
}

/**
* @return The class loader which is associated with the current driver.
* @return The class loader which is associated with the current job.
*/
public ClassLoader getCurrentClassLoader() {
return currentClassLoader;
Expand Down
26 changes: 13 additions & 13 deletions java/runtime/src/main/java/org/ray/runtime/config/RayConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ public class RayConfig {
public final WorkerMode workerMode;
public final RunMode runMode;
public final Map<String, Double> resources;
public final UniqueId driverId;
public final UniqueId jobId;
public final String logDir;
public final boolean redirectOutput;
public final List<String> libraryPath;
Expand All @@ -53,7 +53,7 @@ public class RayConfig {
public final String rayletSocketName;
public final List<String> rayletConfigParameters;

public final String driverResourcePath;
public final String jobResourcePath;
public final String pythonWorkerCommand;

/**
Expand Down Expand Up @@ -105,12 +105,12 @@ public RayConfig(Config config) {
resources.put("CPU", numCpu * 1.0);
}
}
// Driver id.
String driverId = config.getString("ray.driver.id");
if (!driverId.isEmpty()) {
this.driverId = UniqueId.fromHexString(driverId);
// Job id.
String jobId = config.getString("ray.job.id");
if (!jobId.isEmpty()) {
this.jobId = UniqueId.fromHexString(jobId);
} else {
this.driverId = UniqueId.randomId();
this.jobId = UniqueId.randomId();
}
// Log dir.
logDir = removeTrailingSlash(config.getString("ray.log-dir"));
Expand Down Expand Up @@ -160,11 +160,11 @@ public RayConfig(Config config) {
rayletConfigParameters.add(parameter);
}

// Driver resource path.
if (config.hasPath("ray.driver.resource-path")) {
driverResourcePath = config.getString("ray.driver.resource-path");
// Job resource path.
if (config.hasPath("ray.job.resource-path")) {
jobResourcePath = config.getString("ray.job.resource-path");
} else {
driverResourcePath = null;
jobResourcePath = null;
}

// Number of threads that execute tasks.
Expand Down Expand Up @@ -205,7 +205,7 @@ public String toString() {
+ ", workerMode=" + workerMode
+ ", runMode=" + runMode
+ ", resources=" + resources
+ ", driverId=" + driverId
+ ", jobId=" + jobId
+ ", logDir='" + logDir + '\''
+ ", redirectOutput=" + redirectOutput
+ ", libraryPath=" + libraryPath
Expand All @@ -220,7 +220,7 @@ public String toString() {
+ ", objectStoreSize=" + objectStoreSize
+ ", rayletSocketName='" + rayletSocketName + '\''
+ ", rayletConfigParameters=" + rayletConfigParameters
+ ", driverResourcePath='" + driverResourcePath + '\''
+ ", jobResourcePath='" + jobResourcePath + '\''
+ ", pythonWorkerCommand='" + pythonWorkerCommand + '\''
+ '}';
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import org.slf4j.LoggerFactory;

/**
* Manages functions by driver id.
* Manages functions by job id.
*/
public class FunctionManager {

Expand All @@ -46,33 +46,33 @@ public class FunctionManager {
RAY_FUNC_CACHE = ThreadLocal.withInitial(WeakHashMap::new);

/**
* Mapping from the driver id to the functions that belong to this driver.
* Mapping from the job id to the functions that belong to this job.
*/
private Map<UniqueId, DriverFunctionTable> driverFunctionTables = new HashMap<>();
private Map<UniqueId, JobFunctionTable> jobFunctionTables = new HashMap<>();

/**
* The resource path which we can load the driver's jar resources.
* The resource path which we can load the job's jar resources.
*/
private String driverResourcePath;
private String jobResourcePath;

/**
* Construct a FunctionManager with the specified driver resource path.
* Construct a FunctionManager with the specified job resource path.
*
* @param driverResourcePath The specified driver resource that can store the driver's
* @param jobResourcePath The specified job resource that can store the job's
* resources.
*/
public FunctionManager(String driverResourcePath) {
this.driverResourcePath = driverResourcePath;
public FunctionManager(String jobResourcePath) {
this.jobResourcePath = jobResourcePath;
}

/**
* Get the RayFunction from a RayFunc instance (a lambda).
*
* @param driverId current driver id.
* @param jobId current job id.
* @param func The lambda.
* @return A RayFunction object.
*/
public RayFunction getFunction(UniqueId driverId, RayFunc func) {
public RayFunction getFunction(UniqueId jobId, RayFunc func) {
JavaFunctionDescriptor functionDescriptor = RAY_FUNC_CACHE.get().get(func.getClass());
if (functionDescriptor == null) {
SerializedLambda serializedLambda = LambdaUtils.getSerializedLambda(func);
Expand All @@ -82,24 +82,24 @@ public RayFunction getFunction(UniqueId driverId, RayFunc func) {
functionDescriptor = new JavaFunctionDescriptor(className, methodName, typeDescriptor);
RAY_FUNC_CACHE.get().put(func.getClass(), functionDescriptor);
}
return getFunction(driverId, functionDescriptor);
return getFunction(jobId, functionDescriptor);
}

/**
* Get the RayFunction from a function descriptor.
*
* @param driverId Current driver id.
* @param jobId Current job id.
* @param functionDescriptor The function descriptor.
* @return A RayFunction object.
*/
public RayFunction getFunction(UniqueId driverId, JavaFunctionDescriptor functionDescriptor) {
DriverFunctionTable driverFunctionTable = driverFunctionTables.get(driverId);
if (driverFunctionTable == null) {
public RayFunction getFunction(UniqueId jobId, JavaFunctionDescriptor functionDescriptor) {
JobFunctionTable jobFunctionTable = jobFunctionTables.get(jobId);
if (jobFunctionTable == null) {
ClassLoader classLoader;
if (Strings.isNullOrEmpty(driverResourcePath)) {
if (Strings.isNullOrEmpty(jobResourcePath)) {
classLoader = getClass().getClassLoader();
} else {
File resourceDir = new File(driverResourcePath + "/" + driverId.toString() + "/");
File resourceDir = new File(jobResourcePath + "/" + jobId.toString() + "/");
Collection<File> files = FileUtils.listFiles(resourceDir,
new RegexFileFilter(".*\\.jar"), DirectoryFileFilter.DIRECTORY);
files.add(resourceDir);
Expand All @@ -111,31 +111,31 @@ public RayFunction getFunction(UniqueId driverId, JavaFunctionDescriptor functio
}
}).collect(Collectors.toList());
classLoader = new URLClassLoader(urlList.toArray(new URL[urlList.size()]));
LOGGER.debug("Resource loaded for driver {} from path {}.", driverId,
LOGGER.debug("Resource loaded for job {} from path {}.", jobId,
resourceDir.getAbsolutePath());
}

driverFunctionTable = new DriverFunctionTable(classLoader);
driverFunctionTables.put(driverId, driverFunctionTable);
jobFunctionTable = new JobFunctionTable(classLoader);
jobFunctionTables.put(jobId, jobFunctionTable);
}
return driverFunctionTable.getFunction(functionDescriptor);
return jobFunctionTable.getFunction(functionDescriptor);
}

/**
* Manages all functions that belong to one driver.
* Manages all functions that belong to one job.
*/
static class DriverFunctionTable {
static class JobFunctionTable {

/**
* The driver's corresponding class loader.
* The job's corresponding class loader.
*/
ClassLoader classLoader;
/**
* Functions per class, per function name + type descriptor.
*/
Map<String, Map<Pair<String, String>, RayFunction>> functions;

DriverFunctionTable(ClassLoader classLoader) {
JobFunctionTable(ClassLoader classLoader) {
this.classLoader = classLoader;
this.functions = new HashMap<>();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ public void notifyUnblocked(TaskId currentTaskId) {
}

@Override
public TaskId generateTaskId(UniqueId driverId, TaskId parentTaskId, int taskIndex) {
public TaskId generateTaskId(UniqueId jobId, TaskId parentTaskId, int taskIndex) {
return TaskId.randomId();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ public interface RayletClient {

void notifyUnblocked(TaskId currentTaskId);

TaskId generateTaskId(UniqueId driverId, TaskId parentTaskId, int taskIndex);
TaskId generateTaskId(UniqueId jobId, TaskId parentTaskId, int taskIndex);

<T> WaitResult<T> wait(List<RayObject<T>> waitFor, int numReturns, int
timeoutMs, TaskId currentTaskId);
Expand Down
Loading

0 comments on commit 62e4b59

Please sign in to comment.