Commit graph

552 commits

Author SHA1 Message Date
Qing Wang
eb29895dbb
[Core] Remove multiple core workers in one process 1/n. (#24147)
This is the 1st PR to remove the code path of multiple core workers in one process. This PR is aiming to remove the flags and APIs related to `num_workers`.
After this PR checking in, we needn't to consider the multiple core workers any longer.

The further following PRs are related to the deeper logic refactor, like eliminating the gap between core worker and core worker process,  removing the logic related to multiple workers from workerpool, gcs and etc.

**BREAK CHANGE**
This PR removes these APIs:
- Ray.wrapRunnable();
- Ray.wrapCallable();
- Ray.setAsyncContext();
- Ray.getAsyncContext();

And the following APIs are not allowed to invoke in a user-created thread in local mode:
- Ray.getRuntimeContext().getCurrentActorId();
- Ray.getRuntimeContext().getCurrentTaskId()

Note that this PR shouldn't be merged to 1.x.
2022-05-19 00:36:22 +08:00
Qing Wang
40774ac219
Minor changes for Java runtime env. (#24840) 2022-05-17 11:33:59 +08:00
Qing Wang
d40fa391a5
[RuntimeEnv][Java] Support runtime env jars for job. (#24725)
This PR supports specifying the  jars(or zip packages) for a job, which are used for all workers for this job.

You can specify jars or zips in the config file of your job:
```yml
ray {
  job {
    runtime-env: {
         "jars": [
            "https://my_host/a.jar",
            "https://my_host/b.jar"
         ]
    }
  }
}
```
or via system properties:
```java
System.setProperty("ray.job.runtime-env.jars.0", "https://my_host/a.jar");
System.setProperty("ray.job.runtime-env.jars.1", "https://my_host/a.jar");
Ray.init();
// all workers of this job will add a.jar and b.jar into the classpath.
```
2022-05-16 15:07:02 +08:00
Kai Yang
f5c6c7d28f
[Core] Allow failing new tasks immediately while the actor is restarting (#22818)
Currently, when an actor has `max_restarts` > 0 and has crashed, the actor will enter RESTARTING state and then ALIVE. Imagine this scenario: an online service provides HTTP service and the proxy actor receives requests, forwards them to worker actors, and replies to clients with the execution results from worker actors.

```
                                                        -> Worker A (actor)
                                                       /
                                                      /
HTTP requests -------> Proxy (actor with HTTP server) ---> Worker B (actor)
                                                      \
                                                       \
                                                        -> ...
```

For each HTTP request, the proxy picks one worker (e.g. worker A) based on some algorithm, sends the request to it, and calls `ray.get()` to wait for the result. If for some reason the picked worker crashed, Ray will restart the actor, and `ray.get()` will throw an error. The proxy may pick another worker (e.g. worker B) and re-send the request to it. This is OK.

But new requests keep coming. The proxy may pick worker A again. But because worker A is still in RESTARTING state, it's not ready to serve requests. `ray.get()` on subsequent requests sent to worker A will hang until worker A is back online (ALIVE state). The proxy won't be able to reschedule these requests to another worker because currently there's no way to know if worker A is alive or not before sending a request. We can't say worker A is not alive just based on whether `ray.get()` hangs either.

To solve this issue, we change the semantics of `max_task_retries`.

* When max_task_retries is 0 (which is the default value), if the callee actor is in the RESTARTING state, subsequently submitted tasks will fail immediately with a RayActorError. Users can catch the RayActorError and implement their own fallback strategies to improve service availability and mitigate service outages.
* When max_task_retries is not 0, subsequently submitted tasks will be queued on the caller side and we only send them to the callee when the callee actor is back to the ALIVE state.

TODO

- [x] Add test cases.
- [ ] Update docs.
- [x] API change review.
2022-05-14 10:48:47 +08:00
Qing Wang
2627c7b5bc
[Core] Use async post instead of PostBlocking for concurrency group executor. (#24293)
Aiming to:
1. addressing the bug about concurrency group, see #19593
2. improving the stability of the ray call latency perf in online applications.

we're proposing using async post instead of `PostBlocking` in threadpool.

Note that since we have already had back pressure in the caller side, I believe this change is safe to merge and it doesn't break any behavior.
2022-05-13 11:30:52 +08:00
Qing Wang
3208cfc167
[Runtime env][Java] Add unit tests for specifying jars for tasks. (#24712)
It seems that we have already supported specifying java jars for normal tasks, this PR only needs to add unit tests for that.
2022-05-13 09:46:20 +08:00
Qing Wang
259661042c
[runtime env] [java] Support jars in runtime env for Java (#24170)
This PR supports setting the jars for an actor in Ray API. The API looks like:
```java
class A {
    public boolean findClass(String className) {
      try {
        Class.forName(className);
      } catch (ClassNotFoundException e) {
        return false;
      }
      return true;
    }
}

RuntimeEnv runtimeEnv = new RuntimeEnv.Builder()
    .addJars(ImmutableList.of("https://github.com/ray-project/test_packages/raw/main/raw_resources/java-1.0-SNAPSHOT.jar"))
    .build();
ActorHandle<A> actor1 = Ray.actor(A::new).setRuntimeEnv(runtimeEnv).remote();
boolean ret = actor1.task(A::findClass, "io.testpackages.Foo").remote().get();
System.out.println(ret); // true
```
2022-05-12 09:34:40 +08:00
Qing Wang
ea6c2d634b
[Java] Shade jackson to avoid conflict. (#24535)
Jackson is a widely-used utility. User from Ant reports the jackson class is conflicted between Ray jar and user's jar.
This PR shade the jackson in Ray jar to avoid the conflict.

Co-authored-by: Kai Yang <kfstorm@outlook.com>
2022-05-07 10:44:31 +08:00
Qing Wang
c5252c5ceb
[Java] Support parallel actor in experimental. (#21701)
For the purpose to provide an alternative option for running multiple actor instances in a Java worker process, and the eventual goal is to remove the original multi-worker-instances in one worker process implementation.  we're proposing supporting parallel actor concept in Java. This feature enables that users could define some homogeneous parallel execution instances in an actor, and all instances hold one thread as the execution backend.

### Introduction

For the following example, we define a parallel actor with  10 parallelism. The backend actor has 10 concurrency groups for the parallel executions, it also means there're 10 threads for that.

We can access the instance by the instance handle, like:
```java
ParallelActorHandle<A> actor = ParallelActor.actor(A::new).setParallelism(10).remote();
ParallelInstance<A> instance = actor.getInstance(/*index=*/ 2);
Preconditions.checkNotNull(instance);
Ray.get(instance.task(A::incr, 1000000).remote()); // print 1000000           

instance = actor.getInstance(/*index=*/ 2);
Preconditions.checkNotNull(instance);
Ray.get(instance.task(A::incr, 2000000).remote().get()); // print 3000000

instance = actor.getInstance(/*index=*/ 3);
Preconditions.checkNotNull(instance);
Ray.get(instance.task(A::incr, 2000000).remote().get()); // print 2000000
```


### Limitation
- It doesn't support concurrency group on a parallel actor yet.

Co-authored-by: Kai Yang <kfstorm@outlook.com>
2022-04-21 22:54:33 +08:00
Qing Wang
77b0015ea0
[Java] Add NO_RESTART and INFINITE_RESTART constants. (#23771) 2022-04-12 10:40:44 +08:00
Qing Wang
e0ea7567c4
Add getJobId API for ActorId (#23770) 2022-04-08 11:30:53 +08:00
Larry
d0b324990f
[Java] Add doc for Ray.get api that throws an exception if it times out (#23666)
Add doc for Ray.get api that throws an exception if it times out

![image](https://user-images.githubusercontent.com/11072802/161364231-4337124d-3141-4334-879c-f88cecc0d818.png)

Co-authored-by: 稚鱼 <lianjunwen.ljw@antgroup.com>
2022-04-02 18:29:19 +08:00
mwtian
bd4d6b7e19
[Java] upgrade protobuf-java version (#23627) 2022-03-31 09:12:58 -07:00
Qing Wang
ef5b9b87d3
[Java] Add set runtime env api for normal task. (#23412)
This PR adds the API `setRuntimeEnv` for submitting a normal task, for the usage:
```java
RuntimeEnv runtimeEnv =
    new RuntimeEnv.Builder()
        .addEnvVar("KEY1", "A")
        .build();

/// Return `A`
Ray.task(RuntimeEnvTest::getEnvVar, "KEY1").setRuntimeEnv(runtimeEnv).remote().get();
```
2022-03-24 15:57:24 +08:00
Qing Wang
160e2ca9f8
[Java] Add per job runtime env env vars. (#23366)
1. Support setting environment variables in runtime env for a job, like:
```yaml
ray : {
  job : {
    runtime-env: {
         // Environment variables to be set on worker processes in current job.
         "env-vars": {
           // key1: "value11"
           // key2: "value22"
         }
      }
   }
}
```
It could be set by system properties before `Ray.init()` as well:
```java
  System.setProperty("ray.job.runtime-env.env-vars.KEY1", "A");
  System.setProperty("ray.job.runtime-env.env-vars.KEY2", "B");

  Ray.init();
```

2. Setting environment variables for an actor will overwrite and merge to the environment variables of job.
```java
System.setProperty("ray.job.runtime-env.env-vars.KEY1", "A");
System.setProperty("ray.job.runtime-env.env-vars.KEY2", "B");

Ray.init();
  
RuntimeEnv runtimeEnv = new RuntimeEnv.Builder().addEnvVar("KEY1", "C").build();

/// actor1 has the env vars: {"KEY1" : "C", "KEY2" : "B"}
ActorHandle<A> actor1 = Ray.actor(A::new).setRuntimeEnv(runtimeEnv).remote();
/// actor2 has the env vars: {"KEY1" : "A", "KEY2" : "B"} 
ActorHandle<A> actor2 = Ray.actor(A::new).remote();
```
2022-03-23 08:00:00 +08:00
Larry
81dcf9ff35
[Placement Group] Make PlacementGroupID generate from JobID (#23175) 2022-03-21 17:09:16 +08:00
Qing Wang
9f3b4921b6
[Java] Add a default config file for log4j2. (#23225)
We add a default config file for java worker to make info logs are able to be printed before `Ray.init()` invoked.
2022-03-17 11:00:21 +08:00
mwtian
391901f86b
[Remove Redis Pubsub 2/n] clean up remaining Redis references in gcs_utils.py (#23233)
Continue to clean up Redis and other related Redis references, for
- gcs_utils.py
- log_monitor.py
- `publish_error_to_driver()`
2022-03-16 19:34:57 -07:00
Qing Wang
f51cb09e02
[Core][Java][Remove JVM FullGC 2/N] Make JVM be aware of in-memory store pressure. (#21441) 2022-03-15 19:25:27 +08:00
Jialing He
0cbbb8c1d0
[runtime env][core] Use Proto message RuntimeEnvInfo between user code and core_worker (#22856) 2022-03-11 22:14:18 +08:00
qicosmos
e4a9517739
[C++ Worker]Python call cpp worker (#22820) 2022-03-10 11:06:14 -08:00
Qing Wang
9aa0b4e89e
[Java] Add transient for cached hashcode of IDs to reduce serialized size. (#22766)
Use `transient` keyword for reducing the serialized size of  ids for transporting.
2022-03-08 14:36:08 +08:00
Simon Mo
0bab8dbfe0
[Serve] Add test for controller managing Java Replica (#22628) 2022-02-28 23:13:56 -08:00
Qing Wang
9572bb717f
[RuntimeEnv] Support setting actor level env vars for Java worker (#22240)
This PR supports setting actor level env vars for Java worker in runtime env.
General API looks like:
```java
RuntimeEnv runtimeEnv = new RuntimeEnv.Builder()
    .addEnvVar("KEY1", "A")
    .addEnvVar("KEY2", "B")
    .addEnvVar("KEY1", "C")  // This overwrites "KEY1" to "C"
    .build();

ActorHandle<A> actor1 = Ray.actor(A::new).setRuntimeEnv(runtimeEnv).remote();
```

If `num-java-workers-per-process` > 1, it will never reuse the worker process except they have the same runtime envs.

Co-authored-by: Qing Wang <jovany.wq@antgroup.com>
2022-02-28 10:58:37 +08:00
Simon Mo
bfb619a127
[xlang] Allow Python to call overloaded methods with differing number of parameters (#21410) 2022-02-24 16:51:38 -08:00
Qing Wang
bf5693e0b1
[Java] Remove GetGcsClient (#22542)
This PR removes GetGcsClient from core worker and gets necessary data in Java worker.
2022-02-23 03:41:32 -08:00
Qing Wang
96924ecfc0
[Java] Add javac.activative dependency for java worker. (#22538)
This PR adds `javac.activative` as Java worker dependency to address the issue that some users need `JAXB`  on >= JDK9.
2022-02-23 16:24:47 +08:00
Qing Wang
49d725b0c7
[Java] Add Java release guideline. (#22288)
Add Java release guideline to help us release Ray Java process.
2022-02-11 14:56:20 +08:00
Stephanie Wang
dcd96ca348
[core] Increment ref count when creating an ObjectRef to prevent object from going out of scope (#22120)
When a Ray program first creates an ObjectRef (via ray.put or task call), we add it with a ref count of 0 in the C++ backend because the language frontend will increment the initial local ref once we return the allocated ObjectID, then delete the local ref once the ObjectRef goes out of scope. Thus, there is a brief window where the object ref will appear to be out of scope.

This can cause problems with async protocols that check whether the object is in scope or not, such as the previous bug fixed in #19910. Now that we plan to enable lineage reconstruction to automatically recover lost objects, this race condition can also be problematic because we use the ref count to decide whether an object needs to be recovered or not.

This PR avoids these race conditions by incrementing the local ref count in the C++ backend when executing ray.put() and task calls. The frontend is then responsible for skipping the initial local ref increment when creating the ObjectRef. This is the same fix used in #19910, but generalized to all initial ObjectRefs.

This is a re-merge for #21719 with a fix for removing the owned object ref if creation fails.
2022-02-08 14:50:50 -08:00
SangBin Cho
6dda196f47
Revert "[core] Increment ref count when creating an ObjectRef to prev… (#22106)
This reverts commit e3af828220.
2022-02-04 00:55:45 -08:00
Stephanie Wang
e3af828220
[core] Increment ref count when creating an ObjectRef to prevent object from going out of scope (#21719)
When a Ray program first creates an ObjectRef (via ray.put or task call), we add it with a ref count of 0 in the C++ backend because the language frontend will increment the initial local ref once we return the allocated ObjectID, then delete the local ref once the ObjectRef goes out of scope. Thus, there is a brief window where the object ref will appear to be out of scope.

This can cause problems with async protocols that check whether the object is in scope or not, such as the previous bug fixed in #19910. Now that we plan to enable lineage reconstruction to automatically recover lost objects, this race condition can also be problematic because we use the ref count to decide whether an object needs to be recovered or not.

This PR avoids these race conditions by incrementing the local ref count in the C++ backend when executing ray.put() and task calls. The frontend is then responsible for skipping the initial local ref increment when creating the ObjectRef. This is the same fix used in #19910, but generalized to all initial ObjectRefs.
2022-02-03 17:31:27 -08:00
Balaji Veeramani
7f1bacc7dc
[CI] Format Python code with Black (#21975)
See #21316 and #21311 for the motivation behind these changes.
2022-01-29 18:41:57 -08:00
Shawn
6603ad450a
[Java] print hang test case name (#21804)
* print hang test case name

* use getFullTestName
2022-01-23 23:56:44 -08:00
Lingxuan Zuo
ec62d7f510
[Streaming]Farewell : remove all of streaming related from ray repo. (#21770)
New repo url is https://github.com/ray-project/mobius

Co-authored-by: 林濯 <lingxuzn.zlx@antgroup.com>
2022-01-23 17:53:41 +08:00
Qing Wang
a37d9a2ec2
[Core] Support default actor lifetime. (#21283)
Support the ability to specify a default lifetime for actors which are not specified lifetime when creating. This is a job level configuration item.
#### API Change
The Python API looks like:
```python
  ray.init(job_config=JobConfig(default_actor_lifetime="detached"))
```

Java API looks like:
```java
  System.setProperty("ray.job.default-actor-lifetime", defaultActorLifetime.name());
  Ray.init();
```

One example usage is:
```python
  ray.init(job_config=JobConfig(default_actor_lifetime="detached"))
  a1 = A.options(lifetime="non_detached").remote()   # a1 is a non-detached actor.
  a2 = A.remote()  # a2 is a non-detached actor.
```

Co-authored-by: Kai Yang <kfstorm@outlook.com>
Co-authored-by: Qing Wang <jovany.wq@antgroup.com>
2022-01-22 12:26:08 +08:00
SangBin Cho
5514711a35
[Part 5] Set actor died error message in ActorDiedError (#20903)
This is the second last PR to improve `ActorDiedError` exception. 

This propagates Actor death cause metadata to the ray error object. In this way, we can raise a better actor died error exception.

After this PR is merged, I will add more metadata to each error message and write a documentation that explains when each error happens. 

TODO
- [x] Fix test failures
- [x] Add unit tests
- [x] Fix Java/cpp cases

Follow up PRs
- Not allowing nullptr for RayErrorInfo input.
2022-01-20 22:11:11 -08:00
Lingxuan Zuo
43ea467896
Ray support internal native deps reused (#21641)
To make other system or internal project reuse ray deps bazel function, we need change this local accessing style to global accessing with ray-project namespace.

Co-authored-by: 林濯 <lingxuzn.zlx@antgroup.com>
2022-01-21 13:56:40 +08:00
Yi Cheng
3c63a8410d
[gcs/ha] Fix java related error when enable redisless ray (#21692)
This PR enables ray java to be able to run without redis. It also fixes java related tests and updated the pipeline.
2022-01-20 13:56:25 -08:00
Rong Ma
f54282147c
[PlacementGroup] Support using any available bundle in java api (#21496)
In python or C++, we can specify the bundle index as -1 to use any available bundle in the placement group. We should also enable it in Java to keep the API consistent across all languages.
2022-01-18 01:58:02 +08:00
Qing Wang
a5cabb324b
Remove streaming deploying process. (#21603)
1. Remove the streaming from deploying to maven central.
2. Remove related streaming stuff from setup.py.
2022-01-17 23:37:48 +08:00
Qing Wang
6f82bff7ff
[Java] Change ActorLifetime API: DEFAULT -> NON_DETACHED (#21639)
This PR changes the enum value `ActorLifetime.DEFAULT` to `ActorLifetime.NON_DETACHED`. In our release versions, `ActorLifetime` was not introduced <= 1.9.2

Co-authored-by: Qing Wang <jovany.wq@antgroup.com>
2022-01-17 18:10:12 +08:00
Qing Wang
2c3be852ab
[Java] Support defining ConcurrencyGroup statically in Java. (#20373)
This PR introduces statically defining ConcurrencyGroup APIs in Java.
We introduce 2 APIs:
1. Introducing `@DefConcurrencyGroup` annotation for an actor class to define a concurrency group statically.
2. Introducing `@UseConcurrencyGroup` annotation for actor methods to define the concurrency group to be used in the method.

Examples are below:

```java
 @DefConcurrencyGroup(name = "io", maxConcurrency = 2)
  @DefConcurrencyGroup(name = "compute", maxConcurrency = 4)
  private static class MyActor {
    @UseConcurrencyGroup(name = "io")
    public long f1() { }

    @UseConcurrencyGroup(name = "io")
    public long f2() { }

    @UseConcurrencyGroup(name = "compute")
    public long f3(int a, int b) { }

    @UseConcurrencyGroup(name = "compute")
    public long f4() { }
  }

ActorHandle<> myActor = Ray.actor(MyActor::new).remote();
myActor.task(MyActor::f1).remote();
myActor.task(MyActor::f2).remote();
myActor.task(MyActor::f3).remote();
myActor.task(MyActor::f4).remote();
```
`MyActor` has 3 concurrency groups: `io` with 2 concurrency, `compute` with 4 concurrency and `default` with 1 concurrency.
f1 and f2 will be executed in `io`, f3 and f4 will be executed in `compute`.
2022-01-17 16:23:10 +08:00
Qing Wang
bb647626cf
[Xlang][Java] Fix Java overrided default method cannot be invoked. (#21491)
In Xlang(Python call Java), a Java method which overrides a `default` method of the super class is not able to be invoked successfully, due to we treat it as overloaded method instead of overrided method. This PR correctly handle it at the case it overrides a `default` method.

Before this PR, the following usage is not able to be invoked from Python -> Java.
```Java
public interface ExampleInterface {
  default String echo(String inp) {
    return inp;
  }
}
public class ExampleImpl implements ExampleInterface {
  @Override
  public String echo(String inp) {
    return inp + " echo";
  }
}
```
```python
/// Invoke it in Python.
cls = ray.java_actor_class("io.ray.serve.util.ExampleImpl")
handle = cls.remote()
print(ray.get(handle.echo.remote("hi")))
```
2022-01-11 23:11:24 +08:00
Qing Wang
57ff13461c
[Java] Use localhost instead of public ip (#21462)
Use localhost ip address instead of public ip for avoid security popups on MacOS.
This also reverts This reverts commit e4542be0d1.
2022-01-11 02:58:22 +08:00
Yi Cheng
8fa9fddaa0
[1/3][kv] move some internal kv py logic into cpp (#21386)
This PR moves the internal kv namespace logic into cpp to reduce logic in python for the following reasons:

- internal kv is used in x-lang so we have to move it to cpp so that all langs can benefit.
- for https://github.com/ray-project/ray/issues/8822 we need to delete resource when job finished in gcs

One extra field about del is also added so that when delete, we are able to delete by prefix instead of just a key
2022-01-07 17:35:06 -08:00
Clark Zinzow
da4cc26449
[CI] Disable Java log rotation test. (#21394) 2022-01-05 14:51:27 -08:00
Philip Pilgerstorfer
8884cf0f4f
[Java] Bump log4j 2.17.0 to 2.17.1 (#21373)
New log4j version fixes vulnerability:
* https://nvd.nist.gov/vuln/detail/CVE-2021-44832
2022-01-05 09:58:48 +08:00
Qing Wang
240e6efe21
[Java] Try to fix flaky NamespaceTest (#21370) 2022-01-05 09:01:34 +08:00
Qing Wang
340fbf53c0
[Java] Support actor handle reference counting. (#21249) 2022-01-01 10:26:22 +08:00
Tao Wang
a78baf4075
[Java]Init gcs client in runtime only if necessary (#21072)
There's a redis connection in gcs client, but most time the gcs client is never used in worker. We can make the initialization lazy to reduce redis connections.
After that, the number of redis connections reduces from 2 to 1 in one core worker.
2021-12-30 15:44:06 +08:00