Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ZOOKEEPER-4747: Add synchronous sync to ease synchronous call chains #2068

Merged
merged 1 commit into from
Sep 10, 2024

Conversation

kezhuw
Copy link
Member

@kezhuw kezhuw commented Sep 26, 2023

Previously, there is no synchronous sync so client has to convert asynchronous sync a bit to fit synchronous call chains. This is apparently unfriendly.

Besides above, in absent of ZOOKEEPER-22, we can't issue a fire and forget asynchronous sync to gain strong consistent. So it becomes crucial for client to have a convenient synchronous sync.

Refs: ZOOKEEPER-1167, ZOOKEEPER-4747

Previously, there is no synchronous `sync` so client has to convert
asynchronous `sync` a bit to fit synchronous call chains. This is
apparently unfriendly.

Besides above, in absent of ZOOKEEPER-22, we can't issue a fire and
forget asynchronous `sync` to gain strong consistent. So it becomes
crucial for client to have a convenient synchronous `sync`.

Refs: ZOOKEEPER-1167, ZOOKEEPER-4747
@kezhuw kezhuw force-pushed the ZOOKEEPER-4747-synchronous-sync branch from f1424cd to 689d70d Compare September 26, 2023 16:58
* @throws InterruptedException If the server transaction is interrupted.
* @throws IllegalArgumentException if an invalid path is specified
*/
public void sync(final String path) throws KeeperException, InterruptedException {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the path argument even needed? My understanding is that the path has no effect.

Copy link
Member Author

@kezhuw kezhuw Sep 27, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeh, it does no take effect since day one of sync. ZOOKEEPER-3414(#1187) proposed to error on no node. I think we should keep it until we have conclusion about that and semantics of sync path. Otherwise, if ZOOKEEPER-3414 is delivered in future, the sync with no path(default to "/") could fail due to chroot is not required to be existed in data tree. At the least, I think it is a separate issue and we should keep consistent between two versions of sync.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand the argument for keeping it consistent, but I also know it's a pain to have API churn. So, it'd be nice for a decision to be reached and to add the new method in just once, rather than add it, and have to change it (or add a confusing overloaded version) later.

Comment on lines +2720 to +2729
final String clientPath = path;
PathUtils.validatePath(clientPath);

final String serverPath = prependChroot(clientPath);

RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.sync);
SyncRequest request = new SyncRequest();
SyncResponse response = new SyncResponse();
request.setPath(serverPath);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be cleaner to refactor this to share code with the asynchronous version, since everything up to here is identical.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess it may not worth the effort.

  • To cover only above code, we probably have to introduce wrapper struct to cover context for submitRequest and queuePacket.
  • To cover also following code, we probably have to introduce an overload to accept all parameters from asynchronous code and a flag to differentiate origin as zk.sync("/", null, null") is valid.

A little duplication probably be a good here.

Comment on lines +2730 to +2732
ReplyHeader r = cnxn.submitRequest(h, request, response, null);
if (r.getErr() != 0) {
throw KeeperException.create(KeeperException.Code.get(r.getErr()), clientPath);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When an asynchronous method and a synchronous version are available, I feel like the easiest implementation of a synchronous version is something like:

  1. Call the asynchronous version, and return a Future
  2. Make the current thread wait on the completion of the Future that was returned in step 1

This implementation might be doing something like that, but it's not obvious as written. If it's doing anything like that, the details might be obscured inside the submitRequest method. That might make this implementation less readable, and therefore harder to maintain. I think using Futures are more intuitive, if it's not too difficult to implement that way.

You actually have a version like this in the test code in this PR. I'm not sure why that couldn't be the implementation here.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think using Futures are more intuitive, if it's not too difficult to implement that way.

You actually have a version like this in the test code in this PR. I'm not sure why that couldn't be the implementation here.

I am positive to using future internally for synchronous api. This should also solve your above concern about "identical code". But I found ZOOKEEPER-4749 in investigation, may be we should wait a minute. I don't want to handle it specially for sole this api.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

while I understand @ctubbsii concerns, actually the patch this way looks simpler

kezhuw added a commit to kezhuw/zookeeper that referenced this pull request May 29, 2024
…us api

Currently, `zookeeper.request.timeout` is only respected in synchronous
api. I think there are should be no much differences between following two.

```java
String createdPath = zk.create("/path", data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
```

```java
CompletableFuture<String> future = new CompletableFuture<>();
zk.create("/path", data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, (rc, path, ctx, name) -> {
    if (rc == 0) {
        future.complete(name);
    } else {
        future.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path));
    }
}, null);
String createdPath = future.join();
```

After this pr, we are able to unify synchronous api through calls to
asynchronous api as [review comments][review-comments] pointed out if we
feel there are too much identical code between synchronous and asynchronous
api.

[review-comments]: apache#2068 (comment)
kezhuw added a commit to kezhuw/zookeeper that referenced this pull request May 29, 2024
…us api

Currently, `zookeeper.request.timeout` is only respected in synchronous
api. I think there are should be no much differences between following two.

```java
String createdPath = zk.create("/path", data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
```

```java
CompletableFuture<String> future = new CompletableFuture<>();
zk.create("/path", data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, (rc, path, ctx, name) -> {
    if (rc == 0) {
        future.complete(name);
    } else {
        future.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path));
    }
}, null);
String createdPath = future.join();
```

After this pr, we are able to unify synchronous api through calls to
asynchronous api as [review comments][review-comments] pointed out if we
feel there are too much identical code between synchronous and asynchronous
api.

[review-comments]: apache#2068 (comment)
@kezhuw
Copy link
Member Author

kezhuw commented Aug 26, 2024

I plan to merge this now as all remaining review comments are either covered or cloud covered by separated jiras now:

@kezhuw kezhuw merged commit 3e2d6f3 into apache:master Sep 10, 2024
@anmolnar
Copy link
Contributor

@kezhuw @eolivelli Are u sure you don't want this in 3.9 release line?
I see it's new API, but since it's not breaking change, we might want to backport it.

@kezhuw
Copy link
Member Author

kezhuw commented Sep 20, 2024

Do we ever guarantee forward compatibility for released patch versions ?

If not, I am positive for this to be backported. The code is simple and the functionality is almost a simple wrapper for asynchronous version.

We just have encountered that netty breaks forward compatibility (#1917 (comment)). :-)

@anmolnar
Copy link
Contributor

Yes, we only have backward compatiblity rules, but not forward compatibility.

kezhuw added a commit that referenced this pull request Sep 21, 2024
…2068)

Previously, there is no synchronous `sync` so client has to convert
asynchronous `sync` a bit to fit synchronous call chains. This is
apparently unfriendly.

Besides above, in absent of ZOOKEEPER-22, we can't issue a fire and
forget asynchronous `sync` to gain strong consistent. So it becomes
crucial for client to have a convenient synchronous `sync`.

Refs: ZOOKEEPER-1167, ZOOKEEPER-4747

(cherry picked from commit 3e2d6f3)
@kezhuw
Copy link
Member Author

kezhuw commented Sep 21, 2024

Backported to branch-3.9 since it is a simple handy wrapper for asynchronous sync. Also, it touches no data part.

cc @anmolnar

@kezhuw kezhuw deleted the ZOOKEEPER-4747-synchronous-sync branch October 14, 2024 02:38
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants