mirror of https://github.com/apache/lucene.git
SOLR-3582: Our ZooKeeper watchers respond to session events as if they are change events, creating undesirable side effects.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1355004 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
93462313c3
commit
306081e328
|
@ -141,6 +141,8 @@ New Features
|
|||
(yonik, Mark Miller, siren)
|
||||
SOLR-3437: Recovery issues a spurious commit to the cluster. (Trym R. Møller via Mark Miller)
|
||||
SOLR-2822: Skip update processors already run on other nodes (hossman)
|
||||
SOLR-3582: Our ZooKeeper watchers respond to session events as if they are change events,
|
||||
creating undesirable side effects. (Trym R. Møller, Mark Miller)
|
||||
|
||||
* SOLR-1566: Transforming documents in the ResponseWriters. This will allow
|
||||
for more complex results in responses and open the door for function queries
|
||||
|
|
|
@ -30,12 +30,12 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.cloud.ZkCmdExecutor;
|
||||
import org.apache.solr.common.cloud.ZooKeeperException;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.ConnectionLossException;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.Watcher.Event.EventType;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -115,6 +115,11 @@ public class LeaderElector {
|
|||
|
||||
@Override
|
||||
public void process(WatchedEvent event) {
|
||||
// session events are not change events,
|
||||
// and do not remove the watcher
|
||||
if (EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
}
|
||||
// am I the next leader?
|
||||
try {
|
||||
checkIfIamLeader(seq, context, true);
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.concurrent.ThreadFactory;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
|
||||
import org.apache.noggit.CharArr;
|
||||
import org.apache.noggit.JSONParser;
|
||||
import org.apache.noggit.JSONWriter;
|
||||
|
@ -42,6 +41,7 @@ import org.apache.solr.common.util.ByteUtils;
|
|||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.Watcher.Event.EventType;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -180,6 +180,11 @@ public class ZkStateReader {
|
|||
|
||||
@Override
|
||||
public void process(WatchedEvent event) {
|
||||
// session events are not change events,
|
||||
// and do not remove the watcher
|
||||
if (EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
}
|
||||
log.info("A cluster state change has occurred");
|
||||
try {
|
||||
|
||||
|
@ -223,6 +228,11 @@ public class ZkStateReader {
|
|||
|
||||
@Override
|
||||
public void process(WatchedEvent event) {
|
||||
// session events are not change events,
|
||||
// and do not remove the watcher
|
||||
if (EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
}
|
||||
log.info("Updating live nodes");
|
||||
try {
|
||||
// delayed approach
|
||||
|
|
Loading…
Reference in New Issue