You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2016/02/23 19:59:29 UTC
lucene-solr git commit: SOLR-8696: Start the Overseer before actions
that need the overseer on init and when reconnecting after zk expiration and
improve init logic.
Repository: lucene-solr
Updated Branches:
refs/heads/master 7d32456ef -> 8ac4fdd6b
SOLR-8696: Start the Overseer before actions that need the overseer on init and when reconnecting after zk expiration and improve init logic.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8ac4fdd6
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8ac4fdd6
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8ac4fdd6
Branch: refs/heads/master
Commit: 8ac4fdd6bb84b225919d30f6073e7dad02aeb0a1
Parents: 7d32456
Author: markrmiller <ma...@apache.org>
Authored: Tue Feb 23 13:59:04 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Tue Feb 23 13:59:04 2016 -0500
----------------------------------------------------------------------
solr/CHANGES.txt | 3 ++
.../org/apache/solr/cloud/ZkController.java | 36 ++++++++------------
2 files changed, 17 insertions(+), 22 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8ac4fdd6/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3afc961..bf548ae 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -215,6 +215,9 @@ Bug Fixes
* SOLR-8497: Merge index does not mark the Directory objects it creates as 'done' and they are retained in the
Directory cache. (Sivlio Sanchez, Mark Miller)
+* SOLR-8696: Start the Overseer before actions that need the overseer on init and when reconnecting after
+ zk expiration and improve init logic. (Scott Blum, Mark Miller)
+
Optimizations
----------------------
* SOLR-7876: Speed up queries and operations that use many terms when timeAllowed has not been
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8ac4fdd6/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 7874e2b..391f69a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -266,11 +266,8 @@ public final class ZkController {
// seems we dont need to do this again...
// Overseer.createClientNodes(zkClient, getNodeName());
-
- cc.cancelCoreRecoveries();
-
- registerAllCoresAsDown(registerOnReconnect, false);
-
+
+ // start the overseer first as following code may need it's processing
if (!zkRunOnly) {
ElectionContext context = new OverseerElectionContext(zkClient,
overseer, getNodeName());
@@ -284,6 +281,10 @@ public final class ZkController {
overseerElector.joinElection(context, true);
}
+ cc.cancelCoreRecoveries();
+
+ registerAllCoresAsDown(registerOnReconnect, false);
+
zkStateReader.createClusterStateWatchersAndUpdate();
// we have to register as live first to pick up docs in the buffer
@@ -620,26 +621,12 @@ public final class ZkController {
private void init(CurrentCoreDescriptorProvider registerOnReconnect) {
try {
- boolean createdWatchesAndUpdated = false;
- Stat stat = zkClient.exists(ZkStateReader.LIVE_NODES_ZKNODE, null, true);
- if (stat != null && stat.getNumChildren() > 0) {
- zkStateReader.createClusterStateWatchersAndUpdate();
- createdWatchesAndUpdated = true;
- publishAndWaitForDownStates();
- }
-
createClusterZkNodes(zkClient);
- createEphemeralLiveNode();
-
- ShardHandler shardHandler;
- UpdateShardHandler updateShardHandler;
- shardHandler = cc.getShardHandlerFactory().getShardHandler();
- updateShardHandler = cc.getUpdateShardHandler();
-
+ // start the overseer first as following code may need it's processing
if (!zkRunOnly) {
overseerElector = new LeaderElector(zkClient);
- this.overseer = new Overseer(shardHandler, updateShardHandler,
+ this.overseer = new Overseer(cc.getShardHandlerFactory().getShardHandler(), cc.getUpdateShardHandler(),
CommonParams.CORES_HANDLER_PATH, zkStateReader, this, cloudConfig);
ElectionContext context = new OverseerElectionContext(zkClient,
overseer, getNodeName());
@@ -647,10 +634,15 @@ public final class ZkController {
overseerElector.joinElection(context, false);
}
- if (!createdWatchesAndUpdated) {
+ zkStateReader.createClusterStateWatchersAndUpdate();
+ Stat stat = zkClient.exists(ZkStateReader.LIVE_NODES_ZKNODE, null, true);
+ if (stat != null && stat.getNumChildren() > 0) {
zkStateReader.createClusterStateWatchersAndUpdate();
+ publishAndWaitForDownStates();
}
+ // Do this last to signal we're up.
+ createEphemeralLiveNode();
} catch (IOException e) {
log.error("", e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,