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 2020/07/16 11:33:06 UTC
[lucene-solr] branch reference_impl updated: @209 - Keep tightening
close.
This is an automated email from the ASF dual-hosted git repository.
markrmiller pushed a commit to branch reference_impl
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
The following commit(s) were added to refs/heads/reference_impl by this push:
new 82eb2a9 @209 - Keep tightening close.
82eb2a9 is described below
commit 82eb2a9e285a4062011e6ae9260e022bc8b004f0
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Thu Jul 16 06:32:51 2020 -0500
@209 - Keep tightening close.
---
.../java/org/apache/solr/cloud/LeaderElector.java | 4 ++++
.../src/java/org/apache/solr/cloud/Overseer.java | 9 ++++----
.../java/org/apache/solr/cloud/SyncStrategy.java | 25 ++++++++++++----------
3 files changed, 23 insertions(+), 15 deletions(-)
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index 4bcc3a0..2ceadf9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -218,6 +218,10 @@ public class LeaderElector {
* @return sequential node number
*/
public int joinElection(ElectionContext context, boolean replacement,boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
+ if (zkClient.isClosed()) {
+ throw new AlreadyClosedException();
+ }
+
context.joinedElectionFired();
final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 2e2a61c..1b3dbe4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -611,14 +611,15 @@ public class Overseer implements SolrCloseable {
}
public synchronized void start(String id, ElectionContext context) {
- MDCLoggingContext.setNode(zkController == null ?
- null :
- zkController.getNodeName());
-
if (getCoreContainer().isShutDown()) {
if (log.isDebugEnabled()) log.debug("Already closed, exiting");
+ return;
}
+ MDCLoggingContext.setNode(zkController == null ?
+ null :
+ zkController.getNodeName());
+
this.id = id;
this.context = context;
diff --git a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
index 73d053e..ad80836 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
@@ -55,9 +55,9 @@ public class SyncStrategy implements Closeable {
private final ShardHandler shardHandler;
private volatile boolean isClosed;
-
- private final HttpClient client;
-
+
+ private volatile ZkController zkController;
+
private static class RecoveryRequest {
ZkNodeProps leaderProps;
String baseUrl;
@@ -66,7 +66,6 @@ public class SyncStrategy implements Closeable {
public SyncStrategy(CoreContainer cc) {
UpdateShardHandler updateShardHandler = cc.getUpdateShardHandler();
- client = updateShardHandler.getDefaultHttpClient();
shardHandler = ((HttpShardHandlerFactory)cc.getShardHandlerFactory()).getShardHandler(cc.getUpdateShardHandler().getDefaultHttpClient());
}
@@ -85,7 +84,7 @@ public class SyncStrategy implements Closeable {
return PeerSync.PeerSyncResult.success();
}
- if (isClosed || zkController.getCoreContainer().isShutDown()) {
+ if (isClosed()) {
log.warn("Closed, skipping sync up.");
return PeerSync.PeerSyncResult.failure();
}
@@ -98,13 +97,17 @@ public class SyncStrategy implements Closeable {
log.error("No UpdateLog found - cannot sync");
return PeerSync.PeerSyncResult.failure();
}
-
+ this.zkController = zkController;
return syncReplicas(zkController, core, leaderProps, peerSyncOnlyWithActive);
}
-
+
+ private boolean isClosed() {
+ return isClosed || (zkController != null && zkController.getCoreContainer().isShutDown());
+ }
+
private PeerSync.PeerSyncResult syncReplicas(ZkController zkController, SolrCore core,
ZkNodeProps leaderProps, boolean peerSyncOnlyWithActive) {
- if (isClosed) {
+ if (isClosed()) {
log.info("We have been closed, won't sync with replicas");
return PeerSync.PeerSyncResult.failure();
}
@@ -125,7 +128,7 @@ public class SyncStrategy implements Closeable {
SolrException.log(log, "Sync Failed", e);
}
try {
- if (isClosed) {
+ if (isClosed()) {
log.info("We have been closed, won't attempt to sync replicas back to leader");
return PeerSync.PeerSyncResult.failure();
}
@@ -152,7 +155,7 @@ public class SyncStrategy implements Closeable {
List<ZkCoreNodeProps> nodes = zkController.getZkStateReader()
.getReplicaProps(collection, shardId,core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
- if (isClosed) {
+ if (isClosed()) {
log.info("We have been closed, won't sync with replicas");
return PeerSync.PeerSyncResult.failure();
}
@@ -182,7 +185,7 @@ public class SyncStrategy implements Closeable {
String shardId, ZkNodeProps leaderProps, CoreDescriptor cd,
int nUpdates) {
- if (isClosed) {
+ if (isClosed()) {
log.info("We have been closed, won't sync replicas to me.");
return;
}