SOLR-8695: Ensure ZK watchers are not triggering our watch logic on connection events and make this handling more consistent.

This commit is contained in:
markrmiller 2016-02-19 14:36:53 -05:00
parent 32fbca6ea7
commit e30d638c51
9 changed files with 29 additions and 23 deletions

View File

@ -184,6 +184,9 @@ Bug Fixes
* SOLR-8694: DistributedMap/Queue can create too many Watchers and some code simplification.
(Scott Blum via Mark Miller)
* SOLR-8695: Ensure ZK watchers are not triggering our watch logic on connection events and
make this handling more consistent. (Scott Blum via Mark Miller)
Optimizations
----------------------

View File

@ -367,8 +367,7 @@ public class LeaderElector {
@Override
public void process(WatchedEvent event) {
// session events are not change events,
// and do not remove the watcher
// session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}

View File

@ -129,11 +129,13 @@ public class OverseerTaskQueue extends DistributedQueue {
@Override
public void process(WatchedEvent event) {
Event.EventType eventType = event.getType();
// None events are ignored
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}
// If latchEventType is not null, only fire if the type matches
LOG.info("{} fired on path {} state {} latchEventType {}", eventType, event.getPath(), event.getState(), latchEventType);
if (eventType != Event.EventType.None && (latchEventType == null || eventType == latchEventType)) {
LOG.info("{} fired on path {} state {} latchEventType {}", event.getType(), event.getPath(), event.getState(), latchEventType);
if (latchEventType == null || event.getType() == latchEventType) {
synchronized (lock) {
this.event = event;
lock.notifyAll();

View File

@ -2365,7 +2365,8 @@ public final class ZkController {
@Override
public void process(WatchedEvent event) {
if (event.getState() == Event.KeeperState.Disconnected || event.getState() == Event.KeeperState.Expired) {
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}

View File

@ -154,6 +154,7 @@ class CdcrBufferStateManager extends CdcrStateManager {
String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
log.info("The CDCR buffer state has changed: {} @ {}:{}", event, collectionName, shard);
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}

View File

@ -137,6 +137,7 @@ class CdcrLeaderStateManager extends CdcrStateManager {
String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
log.debug("The leader state has changed: {} @ {}:{}", event, collectionName, shard);
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}

View File

@ -154,6 +154,7 @@ class CdcrProcessStateManager extends CdcrStateManager {
String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
log.info("The CDCR process state has changed: {} @ {}:{}", event, collectionName, shard);
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}

View File

@ -248,6 +248,10 @@ public final class ZookeeperInfoHandler extends RequestHandlerBase {
*/
@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;
}
synchronized (this) {
cachedCollections = null;
}

View File

@ -306,8 +306,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
// session events are not change events,
// and do not remove the watcher
// session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@ -359,8 +358,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
// session events are not change events,
// and do not remove the watcher
// session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@ -797,15 +795,14 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
if (!interestingCollections.contains(coll)) {
// This collection is no longer interesting, stop watching.
LOG.info("Uninteresting collection [{}]", coll);
// session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
// session events are not change events,
// and do not remove the watcher
if (EventType.None.equals(event.getType())) {
if (!interestingCollections.contains(coll)) {
// This collection is no longer interesting, stop watching.
LOG.info("Uninteresting collection {}", coll);
return;
}
@ -852,8 +849,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
// session events are not change events,
// and do not remove the watcher
// session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@ -890,8 +886,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
// session events are not change events,
// and do not remove the watcher
// session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@ -924,8 +919,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
// session events are not change events,
// and do not remove the watcher
// session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}