Skip to content

Commit

Permalink
CURATOR-729: Fix PersistentWatcher dead loop after curator closed (#520)
Browse files Browse the repository at this point in the history
The dead loop is multifold:
1. `CuratorFramework::watchers` does not check `CuratorFrameworkState`
   as `getData` or others do.
2. `PersistentWatcher` loops itself through `reset` in `BackgroundCallback`.
3. Callback in `inBackground(callback).forPath(path)` is invoked
   synchronously.

This commit enforces `CuratorFrameworkState` checking also to `watchers`,
`watches`, `sync`, `reconfig` and `getConfig`.

Additionally, this commit will issue `KeeperState.Closed` to listeners
of `PersistentWatcher` when curator get closed. This is not required to
fix CURATOR-729, but will make the closing behavior consistent with
ZooKeeper. Also, I think it is good for asynchronous `Watcher`.

Refs: CURATOR-529, CURATOR-673
  • Loading branch information
kezhuw authored Jan 16, 2025
1 parent 3ccbad7 commit 914f2f7
Show file tree
Hide file tree
Showing 4 changed files with 107 additions and 5 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.curator.framework.api;

public class CuratorClosedException extends IllegalStateException {
public CuratorClosedException() {
super("Expected state [STARTED] was [STOPPED]");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import org.apache.curator.framework.api.ACLProvider;
import org.apache.curator.framework.api.CompressionProvider;
import org.apache.curator.framework.api.CreateBuilder;
import org.apache.curator.framework.api.CuratorClosedException;
import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
import org.apache.curator.framework.api.CuratorListener;
Expand Down Expand Up @@ -462,11 +463,15 @@ public String getNamespace() {

private void checkState() {
CuratorFrameworkState state = getState();
Preconditions.checkState(
state == CuratorFrameworkState.STARTED,
"Expected state [%s] was [%s]",
CuratorFrameworkState.STARTED,
state);
switch (state) {
case STARTED:
return;
case STOPPED:
throw new CuratorClosedException();
default:
String msg = String.format("Expected state [%s] was [%s]", CuratorFrameworkState.STARTED, state);
throw new IllegalStateException(msg);
}
}

@Override
Expand Down Expand Up @@ -525,11 +530,13 @@ public SetACLBuilder setACL() {

@Override
public ReconfigBuilder reconfig() {
checkState();
return new ReconfigBuilderImpl(this);
}

@Override
public GetConfigBuilder getConfig() {
checkState();
return new GetConfigBuilderImpl(this);
}

Expand Down Expand Up @@ -577,11 +584,13 @@ public void sync(String path, Object context) {

@Override
public SyncBuilder sync() {
checkState();
return new SyncBuilderImpl(this);
}

@Override
public RemoveWatchesBuilder watches() {
checkState();
return new RemoveWatchesBuilderImpl(this);
}

Expand All @@ -590,6 +599,7 @@ public WatchesBuilder watchers() {
Preconditions.checkState(
zookeeperCompatibility.hasPersistentWatchers(),
"watchers() is not supported in the ZooKeeper library and/or server being used. Use watches() instead.");
checkState();
return new WatchesBuilderImpl(this);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,15 @@
import java.util.concurrent.atomic.AtomicReference;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.BackgroundCallback;
import org.apache.curator.framework.api.CuratorClosedException;
import org.apache.curator.framework.api.CuratorEventType;
import org.apache.curator.framework.listen.Listenable;
import org.apache.curator.framework.listen.StandardListenerManager;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.curator.utils.ThreadUtils;
import org.apache.zookeeper.AddWatchMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -77,6 +80,11 @@ public PersistentWatcher(CuratorFramework client, String basePath, boolean recur
public void start() {
Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Already started");
client.getConnectionStateListenable().addListener(connectionStateListener);
client.getCuratorListenable().addListener(((ignored, event) -> {
if (event.getType() == CuratorEventType.CLOSING) {
onClientClosed();
}
}));
reset();
}

Expand All @@ -97,6 +105,13 @@ public void close() {
}
}

private void onClientClosed() {
if (state.compareAndSet(State.STARTED, State.CLOSED)) {
WatchedEvent event = new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.Closed, null);
watcher.process(event);
}
}

/**
* Container for setting listeners
*
Expand Down Expand Up @@ -135,6 +150,8 @@ private void reset() {
.inBackground(callback)
.usingWatcher(watcher)
.forPath(basePath);
} catch (CuratorClosedException ignored) {
onClientClosed();
} catch (Exception e) {
log.error("Could not reset persistent watch at path: " + basePath, e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@
package org.apache.curator.framework.recipes.watch;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.framework.state.ConnectionState;
Expand All @@ -46,6 +48,53 @@ public void testConnectionLost() throws Exception {
internalTest(false);
}

@Test
public void testConcurrentClientClose() throws Exception {
BlockingQueue<WatchedEvent> events = new LinkedBlockingQueue<>();

// given: started curator client
CuratorFramework client = CuratorFrameworkFactory.newClient(
server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
client.start();

// given: started persistent watcher
PersistentWatcher persistentWatcher = new PersistentWatcher(client, "/top/main", true);
persistentWatcher.getListenable().addListener(events::add);
persistentWatcher.start();

// when: curator client closed
client.close();

// then: listener get Closed notification
WatchedEvent event = events.poll(5, TimeUnit.SECONDS);
assertNotNull(event);
assertEquals(Watcher.Event.EventType.None, event.getType());
assertEquals(Watcher.Event.KeeperState.Closed, event.getState());
}

@Test
public void testAfterClientClose() throws Exception {
BlockingQueue<WatchedEvent> events = new LinkedBlockingQueue<>();

// given: closed client
CuratorFramework client = CuratorFrameworkFactory.newClient(
server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
client.start();
client.close();

// when: start persistent watcher
try (PersistentWatcher persistentWatcher = new PersistentWatcher(client, "/top/main", true)) {
persistentWatcher.getListenable().addListener(events::add);
persistentWatcher.start();
}

// then: listener get Closed notification
WatchedEvent event = events.poll(5, TimeUnit.SECONDS);
assertNotNull(event);
assertEquals(Watcher.Event.EventType.None, event.getType());
assertEquals(Watcher.Event.KeeperState.Closed, event.getState());
}

private void internalTest(boolean recursive) throws Exception {
try (CuratorFramework client = CuratorFrameworkFactory.newClient(
server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1))) {
Expand Down

0 comments on commit 914f2f7

Please sign in to comment.