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

SOLR-9659: Add DataWatch API #118

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
34 changes: 34 additions & 0 deletions solr/solrj/src/java/org/apache/solr/common/cloud/ChildWatch.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.solr.common.cloud;

import java.util.List;

/**
* Callback interface for notifications of child-node changes in zookeeper state
*/
public interface ChildWatch {

/**
* Called when the children of a watched node change
* @param children the new children
* @return true if the watch should be maintained
*/
boolean onChanged(List<String> children);

}
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,8 @@ public void update(SolrZooKeeper keeper) {
closeKeeper(keeper);
throw new RuntimeException(t);
}

client.onConnect();

if (onReconnect != null) {
onReconnect.command();
Expand Down
33 changes: 33 additions & 0 deletions solr/solrj/src/java/org/apache/solr/common/cloud/DataWatch.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.solr.common.cloud;

/**
* Callback interface for notifications of changes in zookeeper state
*/
public interface DataWatch {

/**
* Called when watched data node is updated
* @param version the data node version
* @param data the new data
* @return true if the node watch should be maintained
*/
boolean onChanged(int version, byte[] data);

}
245 changes: 245 additions & 0 deletions solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,10 +30,16 @@
import java.lang.invoke.MethodHandles;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Pattern;
import java.util.stream.Collectors;

import org.apache.commons.io.FileUtils;
import org.apache.solr.client.solrj.SolrServerException;
Expand Down Expand Up @@ -85,6 +91,10 @@ public class SolrZkClient implements Closeable {
private ZkACLProvider zkACLProvider;
private String zkServerAddress;

private final ConcurrentHashMap<String, Set<DataWatch>> dataWatches = new ConcurrentHashMap<>();
private final ConcurrentHashMap<String, Set<ChildWatch>> childWatches = new ConcurrentHashMap<>();
private final ConcurrentHashMap<String, Integer> versions = new ConcurrentHashMap<>();

public int getZkClientTimeout() {
return zkClientTimeout;
}
Expand Down Expand Up @@ -229,6 +239,241 @@ protected ZkACLProvider createZkACLProvider() {
return new DefaultZkACLProvider();
}

private class NodeWatcher implements Watcher {

final String node;

private NodeWatcher(String node) {
this.node = node;
}

@Override
public void process(WatchedEvent event) {
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}
readNode(node, this);
}
}

/**
* Watch the data at a particular ZK node.
*
* Whenever data on the node changes, the new data is reported back to a
* {@link DataWatch} object. The watch is also notified when a client
* reconnects after a session expiry.
*
* Multiple watchers added to a single node will share ZK watchers.
*
* @param node the node to watch
* @param watch a {@link DataWatch} object to notify
*/
public void addDataWatch(String node, DataWatch watch) {
AtomicBoolean addWatch = new AtomicBoolean(false);
dataWatches.compute(node, (n, s) -> {
if (s == null) {
s = ConcurrentHashMap.newKeySet();
addWatch.set(true);
}
s.add(watch);
return s;
});

readNode(node, addWatch.get() ? new NodeWatcher(node) : null);

}

/**
* Read the data at the supplied ZNode, and notify all registered watches
* @param node the node to read
*/
public void forceUpdateWatch(String node) {
readNode(node, null);
}

public Map<String, Integer> getWatchedVersions() {
return Collections.unmodifiableMap(versions);
}

private void readNode(String node, Watcher watcher) {
Stat stat = new Stat();
try {
while (true) {
try {
byte[] data = getData(node, watcher, stat, true);
notifyDataWatches(node, stat.getVersion(), data);
return;
} catch (KeeperException.NoNodeException e) {
stat = exists(node, watcher, true);
if (stat == null) {
notifyDataWatches(node, -1, null);
return;
}
}
}
}
catch (InterruptedException e) {
if (isClosed == false)
log.warn("Interrupted while setting watch on node {}", node);
Thread.currentThread().interrupt();
}
catch (KeeperException e) {
if (isClosed == false)
log.error("Error setting watch on node {}: {}", node, e.getMessage());
}
}

/**
* Remove a {@link DataWatch} from a particular ZK node
* @param node the node to stop watching
* @param watch the watcher to remove
*/
public void removeDataWatch(String node, DataWatch watch) {
dataWatches.compute(node, (n, s) -> {
s.remove(watch);
if (s.size() == 0)
return null;
return s;
});
}

// lock to force linearization of notifications
private final Object notificationsLock = new Object();

private void notifyDataWatches(String node, int version, byte[] data) {
synchronized (notificationsLock) {
versions.put(node, version);
Set<DataWatch> watchesToRemove = dataWatches.getOrDefault(node, Collections.emptySet())
.stream()
.filter(watch -> watch.onChanged(version, data) == false)
.collect(Collectors.toSet());
for (DataWatch watch : watchesToRemove) {
removeDataWatch(node, watch);
}
}
}

private class ChildWatcher implements Watcher {

final String node;

private ChildWatcher(String node) {
this.node = node;
}

@Override
public void process(WatchedEvent event) {
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}
readChildren(node, this);
}
}

/**
* Watch the children of a particular ZK node.
*
* Whenever the children of the node change, the new child set is reported back to a
* {@link ChildWatch} object. The watch is also notified when a client
* reconnects after a session expiry.
*
* Multiple watchers added to a single node will share ZK watchers.
*
* @param node the node to watch
* @param watch a {@link ChildWatch} object to notify
*/
public void addChildWatch(String node, ChildWatch watch) {
AtomicBoolean addWatch = new AtomicBoolean(false);
childWatches.compute(node, (n, s) -> {
if (s == null) {
s = ConcurrentHashMap.newKeySet();
addWatch.set(true);
}
s.add(watch);
return s;
});

readChildren(node, addWatch.get() ? new ChildWatcher(node) : null);

}

/**
* Read the children of a node from ZK, notifying all registered watchers
* @param node the node to check
*/
public void forceUpdateChildren(String node) {
readChildren(node, null);
}

private void readChildren(String node, Watcher watcher) {
try {
while (true) {
try {
List<String> children = getChildren(node, watcher, true);
notifyChildWatches(node, children);
return;
} catch (KeeperException.NoNodeException e) {
Stat stat = exists(node, watcher, true);
if (stat == null) {
notifyChildWatches(node, Collections.emptyList());
return;
}
}
}
}
catch (InterruptedException e) {
if (isClosed == false)
log.warn("Interrupted while setting watch on node {}", node);
Thread.currentThread().interrupt();
}
catch (KeeperException e) {
if (isClosed == false)
log.error("Error setting watch on node {}: {}", node, e.getMessage());
}
}

/**
* Remove a {@link ChildWatch} from a particular ZK node
* @param node the node to stop watching
* @param watch the watcher to remove
*/
public void removeChildWatch(String node, ChildWatch watch) {
dataWatches.compute(node, (n, s) -> {
s.remove(watch);
if (s.size() == 0)
return null;
return s;
});
}

private synchronized void notifyChildWatches(String node, List<String> children) {

Set<ChildWatch> watchesToRemove = childWatches.getOrDefault(node, Collections.emptySet())
.stream()
.filter(watch -> watch.onChanged(children) == false)
.collect(Collectors.toSet());
for (ChildWatch watch : watchesToRemove) {
removeChildWatch(node, watch);
}
}

/**
* Called by the {@link ConnectionManager} when a session has been established
*
* For internal use only
*/
synchronized void onConnect() {
// re-establish all data watches
for (String node : dataWatches.keySet()) {
readNode(node, new NodeWatcher(node));
}
for (String node : childWatches.keySet()) {
readChildren(node, new ChildWatcher(node));
}
}

/**
* Returns true if client is connected
*/
Expand Down
Loading