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/11/09 04:08:03 UTC
[lucene-solr] 01/04: @1101 Overseer overhaul wip.
This is an automated email from the ASF dual-hosted git repository.
markrmiller pushed a commit to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
commit ce9077d401c6840d5ee59a4a22fef18bd0c04984
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Sun Nov 8 20:47:13 2020 -0600
@1101 Overseer overhaul wip.
---
.../stream/AnalyticsShardRequestManager.java | 3 +-
.../solr/prometheus/scraper/SolrCloudScraper.java | 4 +-
.../prometheus/scraper/SolrCloudScraperTest.java | 4 +-
.../org/apache/solr/cloud/CloudDescriptor.java | 15 -
.../src/java/org/apache/solr/cloud/CloudUtil.java | 14 +-
.../java/org/apache/solr/cloud/DistributedMap.java | 25 +-
.../org/apache/solr/cloud/ElectionContext.java | 6 +-
.../java/org/apache/solr/cloud/LeaderElector.java | 147 ++--
.../src/java/org/apache/solr/cloud/Overseer.java | 909 ++++++++++++---------
.../OverseerCollectionConfigSetProcessor.java | 65 +-
.../cloud/OverseerConfigSetMessageHandler.java | 9 +-
.../apache/solr/cloud/OverseerElectionContext.java | 44 +-
.../solr/cloud/OverseerTaskExecutorTask.java | 157 ++++
.../apache/solr/cloud/OverseerTaskProcessor.java | 28 +-
.../org/apache/solr/cloud/OverseerTaskQueue.java | 248 +++---
.../solr/cloud/RecoveringCoreTermWatcher.java | 10 +-
.../org/apache/solr/cloud/RecoveryStrategy.java | 123 +--
.../solr/cloud/ShardLeaderElectionContext.java | 175 ++--
.../solr/cloud/ShardLeaderElectionContextBase.java | 14 +-
.../java/org/apache/solr/cloud/SyncStrategy.java | 89 +-
.../java/org/apache/solr/cloud/ZkController.java | 662 +++++----------
.../org/apache/solr/cloud/ZkDistributedQueue.java | 730 +++++++----------
.../java/org/apache/solr/cloud/ZkShardTerms.java | 2 +-
.../solr/cloud/api/collections/AddReplicaCmd.java | 312 ++++---
.../solr/cloud/api/collections/AliasCmd.java | 2 +-
.../apache/solr/cloud/api/collections/Assign.java | 97 +--
.../solr/cloud/api/collections/BackupCmd.java | 19 +-
.../solr/cloud/api/collections/CreateAliasCmd.java | 2 +-
.../cloud/api/collections/CreateCollectionCmd.java | 283 +++----
.../solr/cloud/api/collections/CreateShardCmd.java | 120 ++-
.../cloud/api/collections/CreateSnapshotCmd.java | 6 +-
.../solr/cloud/api/collections/DeleteAliasCmd.java | 2 +-
.../cloud/api/collections/DeleteCollectionCmd.java | 43 +-
.../solr/cloud/api/collections/DeleteNodeCmd.java | 13 +-
.../cloud/api/collections/DeleteReplicaCmd.java | 254 +++---
.../solr/cloud/api/collections/DeleteShardCmd.java | 80 +-
.../cloud/api/collections/DeleteSnapshotCmd.java | 6 +-
.../api/collections/MaintainRoutedAliasCmd.java | 2 +-
.../solr/cloud/api/collections/MigrateCmd.java | 31 +-
.../cloud/api/collections/ModifyCollectionCmd.java | 28 +
.../solr/cloud/api/collections/MoveReplicaCmd.java | 220 ++---
.../OverseerCollectionMessageHandler.java | 223 +++--
.../cloud/api/collections/OverseerRoleCmd.java | 3 +-
.../cloud/api/collections/OverseerStatusCmd.java | 18 +-
.../api/collections/ReindexCollectionCmd.java | 6 +-
.../solr/cloud/api/collections/RenameCmd.java | 2 +-
.../solr/cloud/api/collections/ReplaceNodeCmd.java | 213 +++--
.../solr/cloud/api/collections/RestoreCmd.java | 33 +-
.../cloud/api/collections/SetAliasPropCmd.java | 2 +-
.../solr/cloud/api/collections/SplitShardCmd.java | 506 ++++++++----
.../solr/cloud/overseer/ClusterStateMutator.java | 33 +-
.../solr/cloud/overseer/CollectionMutator.java | 19 +-
.../apache/solr/cloud/overseer/NodeMutator.java | 10 +-
.../apache/solr/cloud/overseer/ReplicaMutator.java | 274 ++-----
.../apache/solr/cloud/overseer/SliceMutator.java | 119 ++-
.../apache/solr/cloud/overseer/ZkStateWriter.java | 443 +++++-----
.../apache/solr/cloud/overseer/ZkWriteCommand.java | 16 +-
.../java/org/apache/solr/core/CoreContainer.java | 108 +--
.../org/apache/solr/core/HdfsDirectoryFactory.java | 2 +-
.../src/java/org/apache/solr/core/PluginInfo.java | 4 -
.../src/java/org/apache/solr/core/SolrCore.java | 12 +-
.../src/java/org/apache/solr/core/ZkContainer.java | 28 +-
.../core/snapshots/CollectionSnapshotMetaData.java | 2 +-
.../solr/core/snapshots/SolrSnapshotsTool.java | 2 +-
.../java/org/apache/solr/handler/IndexFetcher.java | 177 ++--
.../apache/solr/handler/ReplicationHandler.java | 33 +-
.../apache/solr/handler/admin/BackupCoreOp.java | 8 +
.../solr/handler/admin/CollectionsHandler.java | 22 +-
.../solr/handler/admin/CoreAdminHandler.java | 1 -
.../solr/handler/admin/CoreAdminOperation.java | 4 +-
.../solr/handler/admin/MetricsHistoryHandler.java | 2 +-
.../apache/solr/handler/admin/PrepRecoveryOp.java | 104 +--
.../solr/handler/admin/RebalanceLeaders.java | 2 -
.../solr/handler/admin/RequestSyncShardOp.java | 5 +-
.../org/apache/solr/handler/admin/SplitOp.java | 13 +-
.../handler/component/RealTimeGetComponent.java | 4 +-
.../org/apache/solr/logging/MDCLoggingContext.java | 2 +-
.../apache/solr/metrics/SolrCoreMetricManager.java | 6 +-
.../metrics/reporters/solr/SolrShardReporter.java | 2 +-
.../org/apache/solr/schema/ManagedIndexSchema.java | 11 +-
.../java/org/apache/solr/schema/SchemaManager.java | 2 +-
.../java/org/apache/solr/servlet/HttpSolrCall.java | 5 +-
.../apache/solr/servlet/SolrDispatchFilter.java | 4 +-
.../org/apache/solr/update/SolrCmdDistributor.java | 33 +-
.../src/java/org/apache/solr/update/UpdateLog.java | 3 +
.../org/apache/solr/update/UpdateShardHandler.java | 1 -
.../processor/DistributedZkUpdateProcessor.java | 72 +-
.../DocExpirationUpdateProcessorFactory.java | 4 +-
.../processor/RoutedAliasUpdateProcessor.java | 2 +-
.../src/java/org/apache/solr/util/ExportTool.java | 12 +-
.../src/java/org/apache/solr/util/SolrCLI.java | 10 +-
.../java/org/apache/solr/util/SolrLogLayout.java | 2 +-
.../apache/solr/HelloWorldSolrCloudTestCase.java | 5 +-
.../test/org/apache/solr/cloud/AddReplicaTest.java | 57 +-
.../cloud/AssignBackwardCompatibilityTest.java | 4 +-
.../apache/solr/cloud/BasicDistributedZkTest.java | 6 +-
.../solr/cloud/ChaosMonkeyShardSplitTest.java | 2 +-
.../org/apache/solr/cloud/CleanupOldIndexTest.java | 1 +
.../solr/cloud/ClusterStateMockUtilTest.java | 2 +-
.../apache/solr/cloud/ClusterStateUpdateTest.java | 10 +-
.../org/apache/solr/cloud/CollectionPropsTest.java | 2 +-
.../apache/solr/cloud/CollectionsAPISolrJTest.java | 80 +-
.../solr/cloud/CreateCollectionCleanupTest.java | 2 +
.../test/org/apache/solr/cloud/DeleteNodeTest.java | 1 +
.../org/apache/solr/cloud/DeleteReplicaTest.java | 81 +-
.../org/apache/solr/cloud/DeleteShardTest.java | 28 +-
.../DistribDocExpirationUpdateProcessorTest.java | 4 +-
.../apache/solr/cloud/DistributedQueueTest.java | 66 +-
.../solr/cloud/DistributedVersionInfoTest.java | 4 +-
.../org/apache/solr/cloud/ForceLeaderTest.java | 3 +-
.../solr/cloud/FullSolrCloudDistribCmdsTest.java | 4 +-
.../solr/cloud/HttpPartitionOnCommitTest.java | 5 +-
.../org/apache/solr/cloud/HttpPartitionTest.java | 18 +-
.../solr/cloud/LeaderElectionContextKeyTest.java | 2 +-
.../org/apache/solr/cloud/LeaderElectionTest.java | 24 +-
.../cloud/LeaderFailoverAfterPartitionTest.java | 7 +-
.../solr/cloud/MissingSegmentRecoveryTest.java | 6 +-
.../solr/cloud/MoveReplicaHDFSFailoverTest.java | 1 -
.../org/apache/solr/cloud/MoveReplicaTest.java | 31 +-
.../org/apache/solr/cloud/NodeMutatorTest.java | 47 +-
.../test/org/apache/solr/cloud/OverseerTest.java | 98 ++-
.../test/org/apache/solr/cloud/RecoveryZkTest.java | 6 +-
.../org/apache/solr/cloud/ReplaceNodeTest.java | 1 +
.../apache/solr/cloud/ReplicationFactorTest.java | 11 +-
.../apache/solr/cloud/ShardRoutingCustomTest.java | 2 +-
.../org/apache/solr/cloud/ShardRoutingTest.java | 10 +-
.../cloud/SharedFSAutoReplicaFailoverTest.java | 2 +-
.../apache/solr/cloud/SolrCloudBridgeTestCase.java | 33 +-
.../apache/solr/cloud/SolrCloudExampleTest.java | 4 +-
.../test/org/apache/solr/cloud/SplitShardTest.java | 20 +-
.../solr/cloud/TestAuthenticationFramework.java | 1 +
.../apache/solr/cloud/TestBaseStatsCacheCloud.java | 4 +-
.../apache/solr/cloud/TestCloudDeleteByQuery.java | 12 +-
.../org/apache/solr/cloud/TestCloudRecovery2.java | 7 +-
.../solr/cloud/TestCloudSearcherWarming.java | 21 +-
.../solr/cloud/TestConfigSetsAPIExclusivity.java | 1 +
.../cloud/TestLeaderElectionWithEmptyReplica.java | 6 +-
.../solr/cloud/TestLocalStatsCacheCloud.java | 1 -
.../solr/cloud/TestOnReconnectListenerSupport.java | 4 +-
.../org/apache/solr/cloud/TestPrepRecovery.java | 11 +-
.../org/apache/solr/cloud/TestPullReplica.java | 2 +-
.../solr/cloud/TestShortCircuitedRequests.java | 19 +-
.../org/apache/solr/cloud/TestTlogReplica.java | 32 +-
.../cloud/TestTolerantUpdateProcessorCloud.java | 14 +-
.../apache/solr/cloud/UnloadDistributedZkTest.java | 8 +-
.../org/apache/solr/cloud/ZkControllerTest.java | 4 +-
.../org/apache/solr/cloud/ZkShardTermsTest.java | 48 +-
.../org/apache/solr/cloud/ZkSolrClientTest.java | 2 +-
.../CollectionsAPIAsyncDistributedZkTest.java | 32 +-
.../CollectionsAPIDistClusterPerZkTest.java | 47 +-
.../solr/cloud/api/collections/ShardSplitTest.java | 11 +-
.../SimpleCollectionCreateDeleteTest.java | 1 +
.../cloud/api/collections/SplitByPrefixTest.java | 143 ++--
.../cloud/overseer/TestClusterStateMutator.java | 6 +-
.../solr/cloud/overseer/ZkStateReaderTest.java | 174 ----
.../solr/cloud/overseer/ZkStateWriterTest.java | 224 -----
.../solr/core/ConfigureRecoveryStrategyTest.java | 3 +-
.../test/org/apache/solr/core/CoreSorterTest.java | 2 +-
.../core/snapshots/TestSolrCloudSnapshots.java | 10 +-
.../solr/handler/TestStressThreadBackup.java | 2 +-
.../solr/handler/admin/IndexSizeEstimatorTest.java | 5 +-
.../handler/admin/ZookeeperStatusHandlerTest.java | 34 +-
.../handler/component/CloudReplicaSourceTest.java | 2 +
.../DistributedQueryComponentOptimizationTest.java | 4 +-
.../handler/component/ShardsWhitelistTest.java | 34 +-
.../component/TestTrackingShardHandlerFactory.java | 8 +-
.../reporters/solr/SolrShardReporterTest.java | 6 +-
.../transform/TestSubQueryTransformerDistrib.java | 1 +
.../apache/solr/search/stats/TestDistribIDF.java | 7 +-
.../processor/RoutedAliasUpdateProcessorTest.java | 2 +-
.../test/org/apache/solr/util/TestExportTool.java | 5 +-
solr/solrj/build.gradle | 2 +
.../solr/client/solrj/cloud/DistributedQueue.java | 12 +-
.../solr/client/solrj/cloud/ReplicaInfo.java | 18 +-
.../client/solrj/impl/BaseCloudSolrClient.java | 84 +-
.../solr/client/solrj/impl/CloudSolrClient.java | 1 +
.../solr/client/solrj/impl/Http2SolrClient.java | 74 +-
.../solr/client/solrj/impl/LBHttp2SolrClient.java | 19 +-
.../solrj/impl/SolrClientNodeStateProvider.java | 2 +-
.../client/solrj/impl/ZkDistribStateManager.java | 2 +-
.../solr/client/solrj/io/sql/StatementImpl.java | 3 +-
.../solrj/io/stream/FeaturesSelectionStream.java | 3 +-
.../client/solrj/io/stream/TextLogitStream.java | 3 +-
.../solr/client/solrj/io/stream/TopicStream.java | 2 +-
.../client/solrj/request/CoreAdminRequest.java | 43 +-
.../solr/client/solrj/request/CoreStatus.java | 4 +
.../src/java/org/apache/solr/common/ParWork.java | 37 +-
.../org/apache/solr/common/cloud/ClusterState.java | 56 +-
.../solr/common/cloud/CompositeIdRouter.java | 5 +-
.../solr/common/cloud/ConnectionManager.java | 4 +-
.../apache/solr/common/cloud/DocCollection.java | 19 +-
.../java/org/apache/solr/common/cloud/Replica.java | 36 +-
.../java/org/apache/solr/common/cloud/Slice.java | 18 +-
.../org/apache/solr/common/cloud/SolrZkClient.java | 199 +++--
.../apache/solr/common/cloud/ZkCmdExecutor.java | 12 +-
.../apache/solr/common/cloud/ZkCoreNodeProps.java | 13 +-
.../org/apache/solr/common/cloud/ZkNodeProps.java | 4 +-
.../apache/solr/common/cloud/ZkStateReader.java | 274 +++++--
.../apache/solr/common/params/CoreAdminParams.java | 2 -
.../solr/common/params/RequiredSolrParams.java | 2 +-
.../solr/common/util/SolrInternalHttpClient.java | 6 +
.../java/org/apache/solr/common/util/TimeOut.java | 2 +-
.../java/org/apache/solr/common/util/Utils.java | 9 +-
.../solrj/impl/BaseSolrClientWireMockTest.java | 20 +-
.../solrj/impl/CloudHttp2SolrClientTest.java | 5 +-
.../impl/CloudHttp2SolrClientWireMockTest.java | 4 +-
.../solr/client/solrj/io/sql/JdbcDriverTest.java | 3 +-
.../solr/client/solrj/io/stream/StreamingTest.java | 2 +-
.../apache/solr/BaseDistributedSearchTestCase.java | 22 +-
.../src/java/org/apache/solr/SolrTestCase.java | 10 +-
.../solr/cloud/AbstractFullDistribZkTestBase.java | 9 +-
.../apache/solr/cloud/MiniSolrCloudCluster.java | 9 +-
.../org/apache/solr/cloud/SolrCloudTestCase.java | 2 +-
.../component/TrackingShardHandlerFactory.java | 2 +-
.../src/resources/logconf/log4j2-startup-debug.xml | 13 +-
versions.props | 3 +-
216 files changed, 4915 insertions(+), 5436 deletions(-)
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java
index 3bc02a1..f6f27db 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java
@@ -119,8 +119,7 @@ public class AnalyticsShardRequestManager {
Collections.shuffle(shuffler, new Random());
Replica rep = shuffler.get(0);
- ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
- String url = zkProps.getCoreUrl();
+ String url = rep.getCoreUrl();
replicaUrls.add(url);
}
} catch (Exception e) {
diff --git a/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java b/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java
index 5880778..eb50f3d 100644
--- a/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java
+++ b/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java
@@ -65,12 +65,12 @@ public class SolrCloudScraper extends SolrScraper {
List<String> coreNames = replicas
.stream()
- .map(Replica::getCoreName)
+ .map(Replica::getName)
.collect(Collectors.toList());
Map<String, Http2SolrClient> coreToClient = replicas
.stream()
- .map(replica -> new Pair<>(replica.getCoreName(), httpSolrClients.get(replica.getBaseUrl())))
+ .map(replica -> new Pair<>(replica.getName(), httpSolrClients.get(replica.getBaseUrl())))
.collect(Collectors.toMap(Pair::first, Pair::second));
return sendRequestsInParallel(coreNames, core -> {
diff --git a/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java b/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java
index ed52b35..0602536 100644
--- a/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java
+++ b/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java
@@ -134,7 +134,7 @@ public class SolrCloudScraperTest extends PrometheusExporterTestBase {
assertEquals(coreCount, allCoreMetrics.size());
for (Map.Entry<String, DocCollection> entry : collectionStates.entrySet()) {
- String coreName = entry.getValue().getReplicas().get(0).getCoreName();
+ String coreName = entry.getValue().getReplicas().get(0).getName();
assertTrue(allCoreMetrics.containsKey(coreName));
List<Collector.MetricFamilySamples> coreMetrics = allCoreMetrics.get(coreName).asList();
assertEquals(1, coreMetrics.size());
@@ -168,7 +168,7 @@ public class SolrCloudScraperTest extends PrometheusExporterTestBase {
Collection<Slice> slices = getCollectionState().getSlices();
Set<String> leaderCoreNames = slices.stream()
- .map(slice -> collection.getLeader(slice.getName()).getCoreName())
+ .map(slice -> collection.getLeader(slice.getName()).getName())
.collect(Collectors.toSet());
for (Collector.MetricFamilySamples.Sample sample : shardLeaderSamples.samples) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
index cec8dbd..4180280 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
@@ -37,7 +37,6 @@ public class CloudDescriptor {
private String collectionName;
private String roles = null;
private Integer numShards;
- private String nodeName = null;
private Map<String,String> collectionParams = new HashMap<>();
private volatile boolean isLeader = false;
@@ -64,9 +63,6 @@ public class CloudDescriptor {
// If no collection name is specified, we default to the core name
this.collectionName = props.getProperty(CoreDescriptor.CORE_COLLECTION, coreName);
this.roles = props.getProperty(CoreDescriptor.CORE_ROLES, null);
- this.nodeName = props.getProperty(CoreDescriptor.CORE_NODE_NAME);
- if (Strings.isNullOrEmpty(nodeName))
- this.nodeName = null;
this.numShards = PropertiesUtil.toInteger(props.getProperty(CloudDescriptor.NUM_SHARDS), null);
String replicaTypeStr = props.getProperty(CloudDescriptor.REPLICA_TYPE);
if (Strings.isNullOrEmpty(replicaTypeStr)) {
@@ -146,16 +142,6 @@ public class CloudDescriptor {
public void setNumShards(int numShards) {
this.numShards = numShards;
}
-
- public String getCoreNodeName() {
- return nodeName;
- }
-
- public void setCoreNodeName(String nodeName) {
- this.nodeName = nodeName;
- if(nodeName==null) cd.getPersistableStandardProperties().remove(CoreDescriptor.CORE_NODE_NAME);
- else cd.getPersistableStandardProperties().setProperty(CoreDescriptor.CORE_NODE_NAME, nodeName);
- }
public void reload(CloudDescriptor reloadFrom) {
if (reloadFrom == null) return;
@@ -166,7 +152,6 @@ public class CloudDescriptor {
if (reloadFrom.getNumShards() != null) {
setNumShards(reloadFrom.getNumShards());
}
- setCoreNodeName(StringUtils.isEmpty(reloadFrom.getCoreNodeName()) ? getCoreNodeName() : reloadFrom.getCoreNodeName());
setLeader(reloadFrom.isLeader);
setHasRegistered(reloadFrom.hasRegistered);
setLastPublished(reloadFrom.getLastPublished());
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
index 9415e5f..0f5cc48 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
@@ -63,10 +63,10 @@ public class CloudUtil {
public static void checkSharedFSFailoverReplaced(CoreContainer cc, CoreDescriptor desc) {
if (desc.getCloudDescriptor() == null) return;
ZkController zkController = cc.getZkController();
- String thisCnn = zkController.getCoreNodeName(desc);
+ String coreName = desc.getName();
String thisBaseUrl = zkController.getBaseUrl();
- log.debug("checkSharedFSFailoverReplaced running for coreNodeName={} baseUrl={}", thisCnn, thisBaseUrl);
+ log.debug("checkSharedFSFailoverReplaced running for coreName={} baseUrl={}", coreName, thisBaseUrl);
// if we see our core node name on a different base url, unload
final DocCollection docCollection = zkController.getClusterState().getCollectionOrNull(desc.getCloudDescriptor().getCollectionName());
@@ -75,11 +75,11 @@ public class CloudUtil {
for (Slice slice : slicesMap.values()) {
for (Replica replica : slice.getReplicas()) {
- String cnn = replica.getName();
+ String replicaName = replica.getName();
String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
- log.debug("compare against coreNodeName={} baseUrl={}", cnn, baseUrl);
+ log.debug("compare against coreName={} baseUrl={}", replicaName, baseUrl);
- if (thisCnn != null && thisCnn.equals(cnn)
+ if (coreName != null && coreName.equals(replicaName)
&& !thisBaseUrl.equals(baseUrl)) {
if (cc.getLoadedCoreNames().contains(desc.getName())) {
try {
@@ -106,12 +106,12 @@ public class CloudUtil {
}
}
- public static boolean replicaExists(ClusterState clusterState, String collection, String shard, String coreNodeName) {
+ public static boolean replicaExists(ClusterState clusterState, String collection, String shard, String coreName) {
DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection != null) {
Slice slice = docCollection.getSlice(shard);
if (slice != null) {
- return slice.getReplica(coreNodeName) != null;
+ return slice.getReplica(coreName) != null;
}
}
return false;
diff --git a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
index 2fa75fe..2bfafe0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
@@ -17,12 +17,17 @@
package org.apache.solr.cloud;
import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.List;
/**
@@ -31,6 +36,8 @@ import java.util.List;
* don't have to be ordered i.e. DistributedQueue.
*/
public class DistributedMap {
+ private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
protected final String dir;
protected SolrZkClient zookeeper;
@@ -38,13 +45,14 @@ public class DistributedMap {
protected static final String PREFIX = "mn-";
public DistributedMap(SolrZkClient zookeeper, String dir) throws KeeperException {
+ log.info("create DistributedMap dir={}", dir);
this.dir = dir;
this.zookeeper = zookeeper;
}
public void put(String trackingId, byte[] data) throws KeeperException, InterruptedException {
- zookeeper.mkdir(dir + "/" + PREFIX + trackingId, data);
+ zookeeper.makePath(dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, false, true);
}
/**
@@ -53,7 +61,7 @@ public class DistributedMap {
*/
public boolean putIfAbsent(String trackingId, byte[] data) throws KeeperException, InterruptedException {
try {
- zookeeper.mkdir(dir + "/" + PREFIX + trackingId, data);
+ zookeeper.makePath(dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, true, true);
return true;
} catch (NodeExistsException e) {
return false;
@@ -61,16 +69,16 @@ public class DistributedMap {
}
public byte[] get(String trackingId) throws KeeperException, InterruptedException {
- return zookeeper.getData(dir + "/" + PREFIX + trackingId, null, null);
+ return zookeeper.getData(dir + "/" + PREFIX + trackingId, null, null, true);
}
public boolean contains(String trackingId) throws KeeperException, InterruptedException {
- return zookeeper.exists(dir + "/" + PREFIX + trackingId);
+ return zookeeper.exists(dir + "/" + PREFIX + trackingId, true);
}
public int size() throws KeeperException, InterruptedException {
Stat stat = new Stat();
- zookeeper.getData(dir, null, stat);
+ zookeeper.getData(dir, null, stat, true);
return stat.getNumChildren();
}
@@ -81,7 +89,7 @@ public class DistributedMap {
*/
public boolean remove(String trackingId) throws KeeperException, InterruptedException {
try {
- zookeeper.delete(dir + "/" + PREFIX + trackingId, -1);
+ zookeeper.delete(dir + "/" + PREFIX + trackingId, -1, true);
} catch (KeeperException.NoNodeException e) {
return false;
}
@@ -94,9 +102,10 @@ public class DistributedMap {
public void clear() throws KeeperException, InterruptedException {
List<String> childNames = zookeeper.getChildren(dir, null, true);
for(String childName: childNames) {
- zookeeper.delete(dir + "/" + childName, -1);
+ zookeeper.delete(dir + "/" + childName, -1, true);
}
-
+//nocommit
+ //zookeeper.delete(paths, true);
}
/**
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index 308f3ba..1441e2e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -20,6 +20,7 @@ import java.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
+import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.zookeeper.KeeperException;
@@ -29,12 +30,12 @@ import org.slf4j.LoggerFactory;
public abstract class ElectionContext implements Closeable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected final String electionPath;
- protected final ZkNodeProps leaderProps;
+ protected final Replica leaderProps;
protected final String id;
protected final String leaderPath;
protected volatile String leaderSeqPath;
- public ElectionContext(final String id, final String electionPath, final String leaderPath, final ZkNodeProps leaderProps) {
+ public ElectionContext(final String id, final String electionPath, final String leaderPath, final Replica leaderProps) {
this.id = id;
this.electionPath = electionPath;
this.leaderPath = leaderPath;
@@ -44,7 +45,6 @@ public abstract class ElectionContext implements Closeable {
}
public void close() {
- leaderSeqPath = null;
assert ObjectReleaseTracker.release(this);
}
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 7b25968..a54aea7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -16,6 +16,7 @@
*/
package org.apache.solr.cloud;
+import java.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Collections;
@@ -54,7 +55,7 @@ import org.slf4j.LoggerFactory;
* starts the whole process over by checking if it's the lowest sequential node, etc.
*
*/
-public class LeaderElector {
+public class LeaderElector implements Closeable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String ELECTION_NODE = "/election";
@@ -63,6 +64,7 @@ public class LeaderElector {
private final static Pattern SESSION_ID = Pattern.compile(".*?/?(.*?-.*?)-n_\\d+");
protected final SolrZkClient zkClient;
+ private final ZkController zkController;
private volatile ElectionContext context;
@@ -70,6 +72,7 @@ public class LeaderElector {
private final Map<ContextKey,ElectionContext> electionContexts;
private final ContextKey contextKey;
+ private volatile boolean isClosed;
// public LeaderElector(SolrZkClient zkClient) {
// this.zkClient = zkClient;
@@ -77,9 +80,10 @@ public class LeaderElector {
// this.electionContexts = new ConcurrentHashMap<>(132, 0.75f, 50);
// }
- public LeaderElector(SolrZkClient zkClient, ContextKey key, Map<ContextKey,ElectionContext> electionContexts) {
+ public LeaderElector(ZkController zkController, ContextKey key, Map<ContextKey,ElectionContext> electionContexts) {
- this.zkClient = zkClient;
+ this.zkClient = zkController.getZkClient();
+ this.zkController = zkController;
this.electionContexts = electionContexts;
this.contextKey = key;
}
@@ -98,11 +102,17 @@ public class LeaderElector {
*/
private boolean checkIfIamLeader(final ElectionContext context, boolean replacement) throws KeeperException,
InterruptedException, IOException {
- log.info("Check if I am leader");
- if (context.isClosed()) {
- throw new AlreadyClosedException();
+ if (isClosed || (zkController != null && zkController.getCoreContainer().isShutDown())) {
+ if (log.isDebugEnabled()) log.debug("Will not checkIfIamLeader, elector is closed");
+ return false;
}
- context.checkIfIamLeaderFired();
+
+ log.info("Check if I am leader {}", context.getClass().getSimpleName());
+
+ ParWork.getRootSharedExecutor().submit(() -> {
+ context.checkIfIamLeaderFired();
+ });
+
boolean checkAgain = false;
// get all other numbers...
@@ -114,6 +124,7 @@ public class LeaderElector {
try {
leaderSeqNodeName = context.leaderSeqPath.substring(context.leaderSeqPath.lastIndexOf('/') + 1);
} catch (NullPointerException e) {
+ if (log.isDebugEnabled()) log.debug("leaderSeqPath has been removed, bailing");
return false;
}
if (!seqs.contains(leaderSeqNodeName)) {
@@ -123,12 +134,21 @@ public class LeaderElector {
if (leaderSeqNodeName.equals(seqs.get(0))) {
// I am the leader
- log.info("I am the leader");
-
- runIamLeaderProcess(context, replacement);
+ log.info("I am the potential leader {}, running leader process", context.leaderProps);
+ ParWork.getRootSharedExecutor().submit(() -> {
+ try {
+ if (isClosed || (zkController != null && zkController.getCoreContainer().isShutDown())) {
+ if (log.isDebugEnabled()) log.debug("Elector is closed, will not try and run leader processes");
+ return;
+ }
+ runIamLeaderProcess(context, replacement);
+ } catch (Exception e) {
+ log.error("", e);
+ }
+ });
} else {
- log.info("I am not the leader - watch the node below me");
+ log.info("I am not the leader - watch the node below me {}", context.getClass().getSimpleName());
String toWatch = seqs.get(0);
for (String node : seqs) {
if (leaderSeqNodeName.equals(node)) {
@@ -140,23 +160,33 @@ public class LeaderElector {
String watchedNode = holdElectionPath + "/" + toWatch;
ElectionWatcher oldWatcher = watcher;
- if (oldWatcher != null) oldWatcher.cancel();
- zkClient.getData(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath, watchedNode, getSeq(context.leaderSeqPath), context), null);
+ if (oldWatcher != null) {
+ oldWatcher.cancel();
+ }
+ zkClient.exists(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath,
+ watchedNode, getSeq(context.leaderSeqPath), context));
if (log.isDebugEnabled()) log.debug("Watching path {} to know if I could be the leader", watchedNode);
} catch (KeeperException.SessionExpiredException e) {
log.error("ZooKeeper session has expired");
throw e;
} catch (KeeperException.NoNodeException e) {
- // the previous node disappeared, check if we are the leader again
+ log.info("the previous node disappeared, check if we are the leader again");
checkAgain = true;
} catch (KeeperException e) {
// we couldn't set our watch for some other reason, retry
- log.warn("Failed setting watch", e);
+ log.info("Failed setting election watch, retrying {} {}", e.getClass().getName(), e.getMessage());
checkAgain = true;
+ } catch (Exception e) {
+ // we couldn't set our watch for some other reason, retry
+ log.info("Failed setting election watch {} {}", e.getClass().getName(), e.getMessage());
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
- }
- return checkAgain;
+ if (checkAgain) {
+ return true;
+ }
+ }
+ return false;
}
// TODO: get this core param out of here
@@ -200,7 +230,7 @@ public class LeaderElector {
}
- public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
+ public boolean joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
return joinElection(context,replacement, false);
}
@@ -213,13 +243,15 @@ public class LeaderElector {
*
* @return sequential node number
*/
- public int joinElection(ElectionContext context, boolean replacement,boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
- if (context.isClosed() || zkClient.isClosed()) {
- if (log.isDebugEnabled()) log.debug("Already closed");
- throw new AlreadyClosedException();
+ public boolean joinElection(ElectionContext context, boolean replacement,boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
+ if (isClosed || (zkController != null && zkController.getCoreContainer().isShutDown())) {
+ if (log.isDebugEnabled()) log.debug("Will not join election, elector is closed");
+ return false;
}
- context.joinedElectionFired();
+ ParWork.getRootSharedExecutor().submit(() -> {
+ context.joinedElectionFired();
+ });
final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
@@ -248,17 +280,9 @@ public class LeaderElector {
zkClient.create(leaderSeqPath, null, CreateMode.EPHEMERAL, false);
}
} else {
- while (true) {
- if (log.isDebugEnabled()) log.debug("create ephem election node {}", shardsElectZkPath + "/" + id + "-n_");
- try {
+ if (log.isDebugEnabled()) log.debug("create ephem election node {}", shardsElectZkPath + "/" + id + "-n_");
leaderSeqPath = zkClient.create(shardsElectZkPath + "/" + id + "-n_", null,
CreateMode.EPHEMERAL_SEQUENTIAL, false);
- break;
- } catch (ConnectionLossException e) {
- log.warn("Connection loss during leader election, trying again ...");
- continue;
- }
- }
}
if (log.isDebugEnabled()) log.debug("Joined leadership election with path: {}", leaderSeqPath);
@@ -288,25 +312,43 @@ public class LeaderElector {
} catch (KeeperException.NoNodeException e) {
// we must have failed in creating the election node - someone else must
// be working on it, lets try again
+ log.info("No node found during election {} " + e.getMessage(), e.getPath());
if (tries++ > 5) {
- context = null;
+ log.error("No node found during election {} " + e.getMessage(), e.getPath());
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
- "", e);
+ "", e);
}
cont = true;
}
}
- while(checkIfIamLeader(context, replacement)) {
- if (context.isClosed()) {
- throw new AlreadyClosedException();
- }
+
+ int seq = getSeq(context.leaderSeqPath);
+
+ boolean tryagain = checkIfIamLeader(context, replacement);
+
+ if (tryagain) {
+ checkIfIamLeader(context, replacement);
}
- return getSeq(context.leaderSeqPath);
+
+// boolean tryagain = false;
+// while (tryagain) {
+// tryagain = checkIfIamLeader(context, replacement);
+// if (tryagain) {
+// Thread.sleep(250);
+// }
+// }
+
+ return false;
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.isClosed = true;
}
private class ElectionWatcher implements Watcher {
- final String myNode,watchedNode;
+ final String myNode, watchedNode;
final ElectionContext context;
private volatile boolean canceled = false;
@@ -319,7 +361,6 @@ public class LeaderElector {
void cancel() {
canceled = true;
-
}
@Override
@@ -329,29 +370,20 @@ public class LeaderElector {
return;
}
if (canceled) {
- log.debug("This watcher is not active anymore {}", myNode);
- try {
- zkClient.delete(myNode, -1);
- } catch (AlreadyClosedException | InterruptedException e) {
- log.info("Already shutting down");
- return;
- } catch (KeeperException.NoNodeException nne) {
- log.info("No znode found to delete at {}", myNode);
- // expected . don't do anything
- } catch (Exception e) {
- log.error("Exception canceling election", e);
- return;
- }
+ if (log.isDebugEnabled()) log.debug("This watcher is not active anymore {}", myNode);
return;
}
try {
// am I the next leader?
- checkIfIamLeader(context, true);
+ boolean tryagain = checkIfIamLeader(context, true);
+ if (tryagain) {
+ checkIfIamLeader(context, true);
+ }
} catch (AlreadyClosedException | InterruptedException e) {
log.info("Already shutting down");
return;
} catch (Exception e) {
- log.error("Exception canceling election", e);
+ log.error("Exception in election", e);
return;
}
}
@@ -372,9 +404,6 @@ public class LeaderElector {
}
void retryElection(ElectionContext context, boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
- if (context.isClosed()) {
- throw new AlreadyClosedException();
- }
ElectionWatcher watcher = this.watcher;
if (electionContexts != null) {
ElectionContext prevContext = electionContexts.put(contextKey, context);
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 45f4ebe..463ac1a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -17,25 +17,20 @@
package org.apache.solr.cloud;
import java.io.Closeable;
-import java.io.File;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.HashSet;
-import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.LongAdder;
import java.util.function.BiConsumer;
import org.apache.lucene.util.Version;
import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.impl.CloudHttp2SolrClient;
import org.apache.solr.client.solrj.impl.Http2SolrClient;
@@ -44,14 +39,8 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
-import org.apache.solr.cloud.overseer.ClusterStateMutator;
-import org.apache.solr.cloud.overseer.CollectionMutator;
-import org.apache.solr.cloud.overseer.NodeMutator;
import org.apache.solr.cloud.overseer.OverseerAction;
-import org.apache.solr.cloud.overseer.ReplicaMutator;
-import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.cloud.overseer.ZkStateWriter;
-import org.apache.solr.cloud.overseer.ZkWriteCommand;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrCloseable;
@@ -60,32 +49,29 @@ import org.apache.solr.common.SolrThread;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ConnectionManager;
import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
-import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.admin.CollectionsHandler;
-import org.apache.solr.handler.component.HttpShardHandler;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.update.UpdateShardHandler;
-import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.CONFIGSETS_ACTION_PREFIX;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+
/**
* <p>Cluster leader. Responsible for processing state updates, node assignments, creating/deleting
* collections, shards, replicas and setting various properties.</p>
@@ -156,7 +142,6 @@ public class Overseer implements SolrCloseable {
public static final String OVERSEER_COLLECTION_MAP_RUNNING = "/overseer/collection-map-running";
- public static final String OVERSEER_QUEUE_WORK = "/overseer/queue-work";
// System properties are used in tests to make them run fast
public static final int STATE_UPDATE_DELAY = ZkStateReader.STATE_UPDATE_DELAY;
@@ -167,13 +152,21 @@ public class Overseer implements SolrCloseable {
public static final String OVERSEER_ELECT = "/overseer/overseer_elect";
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- private volatile OverseerElectionContext context;
+
private volatile boolean closeAndDone;
+ private volatile boolean initedHttpClient = false;
+ private volatile QueueWatcher queueWatcher;
+ private volatile CollectionWorkQueueWatcher collectionQueueWatcher;
public boolean isDone() {
return closeAndDone;
}
+
+ // public ExecutorService getTaskExecutor() {
+// return taskExecutor;
+// }
+
private static class StringBiConsumer implements BiConsumer<String, Object> {
boolean firstPair = true;
@@ -201,6 +194,7 @@ public class Overseer implements SolrCloseable {
//queue where everybody can throw tasks
private final ZkDistributedQueue stateUpdateQueue;
+
private volatile boolean isClosed = false;
public ClusterStateUpdater(final ZkStateReader reader, final String myId, Stats zkStats) {
@@ -219,114 +213,92 @@ public class Overseer implements SolrCloseable {
MDCLoggingContext.setNode(zkController.getNodeName() );
try {
- log.info("Starting to work on the main queue : {}", LeaderElector.getNodeName(myId));
- ZkStateWriter zkStateWriter = new ZkStateWriter(reader, stats);;
- try {
- reader.forciblyRefreshAllClusterStateSlow();
- } catch (KeeperException e) {
- log.error("", e);
- return;
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- return;
- }
- ClusterState clusterState = reader.getClusterState();
- assert clusterState != null;
+ log.info("Starting to work on the main queue : {}", LeaderElector.getNodeName(myId));
+
+// ClusterState clusterState = reader.getClusterState();
+// assert clusterState != null;
- while (!checkClosed()) {
- if (log.isDebugEnabled()) log.debug("Start of Overseer loop ...");
+ // we write updates in batch, but if an exception is thrown when writing new clusterstate,
+ // we do not sure which message is bad message, therefore we will re-process node one by one
- LinkedList<Pair<String, byte[]>> queue = null;
- try {
- // We do not need to filter any nodes here cause all processed nodes are removed once we flush clusterstate
+ while (!this.isClosed) {
+
+ LinkedList<Pair<String,byte[]>> queue = null;
+ try {
+ // We do not need to filter any nodes here cause all processed nodes are removed once we flush clusterstate
+ queue = new LinkedList<>(stateUpdateQueue.peekElements(1000, 3000L, (x) -> true));
- long wait = 1000;
- queue = new LinkedList<>(stateUpdateQueue.peekElements(1000, wait, (x) -> x.startsWith(OverseerTaskQueue.RESPONSE_PREFIX)));
- } catch (AlreadyClosedException e) {
- if (isClosed()) {
- log.info("Overseer closed (AlreadyClosedException), exiting loop");
+ log.info("peeking at the status update queue {}", printQueue(queue));
+ } catch (InterruptedException | AlreadyClosedException e) {
+ ParWork.propagateInterrupt(e, true);
+ log.error("Unexpected error in Overseer state update loop", e);
return;
- }
- return;
- } catch (KeeperException.SessionExpiredException e) {
- log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
- return;
- } catch (Exception e) {
- ParWork.propagateInterrupt("Unexpected error in Overseer state update loop", e, true);
- log.error("Unexpected error in Overseer state update loop", e);
- if (!isClosed()) {
- continue;
- } else {
+ } catch (KeeperException.SessionExpiredException e) {
+ log.error("run()", e);
+
+ log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
+ return;
+ } catch (Exception e) {
+ ParWork.propagateInterrupt(e, true);
+ log.error("Unexpected error in Overseer state update loop", e);
return;
}
- }
- try {
- Set<String> processedNodes = new HashSet<>();
- TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS, TimeSource.NANO_TIME);
- while (queue != null && !queue.isEmpty()) {
- if (isClosed()) {
- log.info("Closing");
- return;
- }
- for (Pair<String, byte[]> head : queue) {
- byte[] data = head.second();
+ try {
+ Set<String> processedNodes = new HashSet<>();
+ while (queue != null && !queue.isEmpty()) {
+ for (Pair<String,byte[]> head : queue) {
+ byte[] data = head.second();
+ if (log.isDebugEnabled()) log.debug("look at node {} data={}", head.first(), head.second() == null ? null : head.second().length);
- if (log.isDebugEnabled()) log.debug("look at node {} data={}", head.first(), head.second() == null ? null : head.second().length);
- if (head.second() != null && head.second().length > 0) {
final ZkNodeProps message = ZkNodeProps.load(data);
if (log.isDebugEnabled()) log.debug("processMessage: queueSize: {}, message = {}", stateUpdateQueue.getZkStats().getQueueLength(), message);
if (log.isDebugEnabled()) log.debug("add processed node: {}, processedNodes = {}", head.first(), stateUpdateQueue.getZkStats().getQueueLength(), processedNodes);
- processedNodes.add(new File(head.first()).getName());
+ processedNodes.add(head.first());
+
// The callback always be called on this thread
- processQueueItem(message, reader.getClusterState(), zkStateWriter, true, null);
- } else {
- log.warn("Found queue item with no data, removing it {} : {}", head.first(), new File(head.first()).getName());
- processedNodes.add(new File(head.first()).getName());
- }
- }
- if (isClosed()) {
- log.info("Overseer closed, exiting loop");
- return;
- }
+ boolean success = processQueueItem(message);
+ if (success) {
+ // nocommit
+ stateUpdateQueue.remove(processedNodes);
+ processedNodes.clear();
+ }
- if (timeout.hasTimedOut()) {
- if (log.isDebugEnabled()) log.debug("timeout, skipping out on tight loop {}", timeout.getInterval(TimeUnit.MILLISECONDS));
- break;
+ }
+ // if (isClosed) break;
+ // if an event comes in the next 100ms batch it together
+ log.info("peekElements");
+ queue = new LinkedList<>(stateUpdateQueue.peekElements(1000, 100, node -> !processedNodes.contains(node)));
}
- // if an event comes in the next *ms batch it together
- int wait = 10;
- if (log.isDebugEnabled()) log.debug("going to peekElements processedNodes={}", processedNodes);
- queue = new LinkedList<>(stateUpdateQueue.peekElements(10, wait, node -> processedNodes.contains(node) || node.startsWith(OverseerTaskQueue.RESPONSE_PREFIX)));
- }
- clusterState = zkStateWriter.writePendingUpdates(clusterState, () -> {
- if (log.isDebugEnabled()) log.debug("clear processedNodes={}", processedNodes);
+ // we should force write all pending updates because the next iteration might sleep until there
+ // are more items in the main queue
+ log.info("writePending updates");
+ writePendingUpdates();
+ // clean work queue
+ log.info("clean work queue");
stateUpdateQueue.remove(processedNodes);
processedNodes.clear();
- });
-
- } catch (InterruptedException | AlreadyClosedException e) {
- ParWork.propagateInterrupt(e, true);
- return;
- } catch (KeeperException.SessionExpiredException e) {
- log.error("run()", e);
-
- log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
- return;
- } catch (Exception e) {
- log.error("Unexpected error in Overseer state update loop, exiting ...", e);
- return;
+ } catch (InterruptedException | AlreadyClosedException e) {
+ Thread.currentThread().interrupt();
+ return;
+ } catch (KeeperException.SessionExpiredException e) {
+ log.error("run()", e);
+
+ log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
+ return;
+ } catch (Exception e) {
+ log.error("", e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
}
- }
- } finally {
+ } finally {
log.info("Overseer Loop exiting : {}", LeaderElector.getNodeName(myId));
- if (!isClosed && !closeAndDone) { // if we have not been closed, close so that we stop the other threads
- Overseer.this.close(true);
+ if (!isClosed) {
+ Overseer.this.close();
}
- itemsQueued.reset();
}
if (log.isDebugEnabled()) {
@@ -363,173 +335,6 @@ public class Overseer implements SolrCloseable {
return false;
}
- private LongAdder itemsQueued = new LongAdder();
-
- private ClusterState processQueueItem(ZkNodeProps message, ClusterState clusterState, ZkStateWriter zkStateWriter,
- boolean enableBatching, ZkStateWriter.ZkWriteCallback callback) throws Exception {
- if (log.isDebugEnabled()) log.debug("Consume state update from queue {}", message);
- // assert clusterState != null;
-
- ClusterState cs = null;
- // if (clusterState.getZNodeVersion() == 0 || clusterState.getZNodeVersion() > lastVersion) {
-
- final String operation = message.getStr(QUEUE_OPERATION);
- if (operation == null) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Message missing " + QUEUE_OPERATION + ":" + message);
- }
- itemsQueued.increment();
- Map<String,DocCollection> updatesToWrite = zkStateWriter.getUpdatesToWrite();
- ClusterState state;
- LinkedHashMap<String,ClusterState.CollectionRef> collStates;
- ClusterState prevState = null;
-
- state = clusterState;
-
- collStates = new LinkedHashMap<>(state.getCollectionStates());
- for (DocCollection docCollection : updatesToWrite.values()) {
- Map<String,Slice> slicesMap = docCollection.getSlicesMap();
- DocCollection oldDoc = state.getCollectionOrNull(docCollection.getName());
- Map<String,Slice> newSlicesMap;
- if (oldDoc != null) {
- newSlicesMap = new HashMap(oldDoc.getSlicesMap());
- } else {
- newSlicesMap = new HashMap();
- }
-
- for (Slice slice : slicesMap.values()) {
- Slice oldSlice = null;
- if (oldSlice != null && oldDoc.getSlicesMap() != null) {
- oldSlice = oldDoc.getSlicesMap().get(slice.getName());
- }
- Map<String,Replica> existingReplicas;
- if (oldSlice != null && oldSlice.getReplicasMap() != null) {
- existingReplicas = new HashMap<>(oldSlice.getReplicasMap());
- } else {
- existingReplicas = new HashMap<>();
- }
-
- for (Replica ereplica : slice.getReplicas()) {
- existingReplicas.put(ereplica.getName(), ereplica);
- }
-
- newSlicesMap.put(slice.getName(), new Slice(slice.getName(), existingReplicas, slice.getProperties(), docCollection.getName()));
- }
- collStates.put(docCollection.getName(), new ClusterState.CollectionRef(
- new DocCollection(docCollection.getName(), newSlicesMap, docCollection.getProperties(), docCollection.getRouter(), docCollection.getZNodeVersion())));
-
- }
- prevState = new ClusterState(state.getLiveNodes(), collStates, state.getZNodeVersion());
-
- List<ZkWriteCommand> zkWriteOps = processMessage(prevState, message, operation);
-
- cs = zkStateWriter.enqueueUpdate(prevState, zkWriteOps, callback);
- // }
- if (log.isDebugEnabled()) log.debug("State update consumed from queue {}", message);
- return cs;
- }
-
- private List<ZkWriteCommand> processMessage(ClusterState clusterState,
- final ZkNodeProps message, final String operation) {
- if (log.isDebugEnabled()) {
- log.debug("processMessage(ClusterState clusterState={}, ZkNodeProps message={}, String operation={}) - start", clusterState, message, operation);
- }
-
- CollectionParams.CollectionAction collectionAction = CollectionParams.CollectionAction.get(operation);
- if (collectionAction != null) {
- switch (collectionAction) {
- case CREATE:
- return Collections.singletonList(new ClusterStateMutator(getSolrCloudManager()).createCollection(clusterState, message));
- case DELETE:
- return Collections.singletonList(new ClusterStateMutator(getSolrCloudManager()).deleteCollection(clusterState, message));
- case CREATESHARD:
- List<ZkWriteCommand> ret = Collections.singletonList(new CollectionMutator(getSolrCloudManager()).createShard(clusterState, message));
- String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
- String shardName = message.getStr(ZkStateReader.SHARD_ID_PROP);
- try {
- // TODO: remove this makePath calls and require making each path individually correctly
- getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/" + shardName, null, CreateMode.PERSISTENT, false);
- // stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect", null, CreateMode.PERSISTENT, false);
- getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName, null, CreateMode.PERSISTENT, false);
- getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName + "/election", null, CreateMode.PERSISTENT, false);
- getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leaders/" + shardName, null, CreateMode.PERSISTENT, false);
- getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/terms/" + shardName, ZkStateReader.emptyJson, CreateMode.PERSISTENT, false);
-
- } catch (KeeperException | IOException | InterruptedException | AlreadyExistsException e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- }
- return ret;
- case DELETESHARD:
- return Collections.singletonList(new CollectionMutator(getSolrCloudManager()).deleteShard(clusterState, message));
- case ADDREPLICA:
- return Collections.singletonList(new SliceMutator(getSolrCloudManager()).addReplica(clusterState, message));
- case ADDREPLICAPROP:
- return Collections.singletonList(new ReplicaMutator(getSolrCloudManager()).addReplicaProperty(clusterState, message));
- case DELETEREPLICAPROP:
- return Collections.singletonList(new ReplicaMutator(getSolrCloudManager()).deleteReplicaProperty(clusterState, message));
- case BALANCESHARDUNIQUE:
- ExclusiveSliceProperty dProp = new ExclusiveSliceProperty(clusterState, message);
- if (dProp.balanceProperty()) {
- String collName = message.getStr(ZkStateReader.COLLECTION_PROP);
- List<ZkWriteCommand> returnList = Collections.singletonList(new ZkWriteCommand(collName, dProp.getDocCollection()));
- if (log.isDebugEnabled()) {
- log.debug("processMessage(ClusterState, ZkNodeProps, String) - end");
- }
- return returnList;
- }
- break;
- case MODIFYCOLLECTION:
- ZkWriteCommand zkwrite = new CollectionMutator(getSolrCloudManager()).modifyCollection(clusterState, message);
- return Collections.singletonList(zkwrite);
- default:
- throw new RuntimeException("unknown operation:" + operation
- + " contents:" + message.getProperties());
- }
- } else {
- OverseerAction overseerAction = OverseerAction.get(operation);
- if (overseerAction == null) {
- throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
- }
- switch (overseerAction) {
- case STATE:
- return Collections.singletonList(new ReplicaMutator(getSolrCloudManager()).setState(clusterState, message));
- case LEADER:
- return Collections.singletonList(new SliceMutator(getSolrCloudManager()).setShardLeader(clusterState, message));
- case DELETECORE:
- return Collections.singletonList(new SliceMutator(getSolrCloudManager()).removeReplica(clusterState, message));
- case ADDROUTINGRULE:
- return Collections.singletonList(new SliceMutator(getSolrCloudManager()).addRoutingRule(clusterState, message));
- case REMOVEROUTINGRULE:
- return Collections.singletonList(new SliceMutator(getSolrCloudManager()).removeRoutingRule(clusterState, message));
- case UPDATESHARDSTATE:
- return Collections.singletonList(new SliceMutator(getSolrCloudManager()).updateShardState(clusterState, message));
-// case QUIT:
-// if (myId.equals(message.get(ID))) {
-// log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId));
-// try {
-// overseerCollectionConfigSetProcessor.close();
-// } catch (IOException e) {
-// log.error("IOException", e);
-// }
-// close();
-// } else {
-// log.warn("Overseer received wrong QUIT message {}", message);
-// }
-// break;
- case DOWNNODE:
- return new NodeMutator().downNode(clusterState, message);
- default:
- throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
- }
- }
-
- List<ZkWriteCommand> returnList = Collections.singletonList(ZkStateWriter.NO_OP);
- if (log.isDebugEnabled()) {
- log.debug("processMessage(ClusterState, ZkNodeProps, String) - end");
- }
- return returnList;
- }
-
@Override
public void close() {
if (log.isDebugEnabled()) {
@@ -542,6 +347,16 @@ public class Overseer implements SolrCloseable {
}
}
+ private String printQueue(LinkedList<Pair<String,byte[]>> queue) {
+
+ StringBuilder sb = new StringBuilder("Queue[");
+ for (Pair<String,byte[]> item : queue) {
+ sb.append(item.first()).append(":").append(ZkNodeProps.load(item.second())).append(", ");
+ }
+ sb.append("]");
+ return sb.toString();
+ }
+
public static class OverseerThread extends SolrThread implements Closeable {
protected volatile boolean isClosed;
@@ -573,11 +388,14 @@ public class Overseer implements SolrCloseable {
}
- private volatile OverseerThread ccThread;
- private volatile OverseerThread updaterThread;
+ // private volatile OverseerThread updaterThread;
+
+// private volatile ExecutorService stateManagmentExecutor;
+//
+// private volatile ExecutorService taskExecutor;
- private volatile OverseerThread triggerThread;
+ private volatile ZkStateWriter zkStateWriter;
private final ZkStateReader reader;
@@ -591,7 +409,7 @@ public class Overseer implements SolrCloseable {
private volatile Stats stats;
private volatile String id;
- private volatile boolean closed;
+ private volatile boolean closed = true;
private volatile boolean systemCollCompatCheck = true;
private final CloudConfig config;
@@ -608,15 +426,16 @@ public class Overseer implements SolrCloseable {
this.zkController = zkController;
this.stats = new Stats();
this.config = config;
-
}
public synchronized void start(String id, ElectionContext context) throws KeeperException {
- doClose();
+ log.info("Staring Overseer");
if (getCoreContainer().isShutDown() || closeAndDone) {
if (log.isDebugEnabled()) log.debug("Already closed, exiting");
return;
}
+
+ doClose();
closed = false;
MDCLoggingContext.setNode(zkController == null ?
@@ -624,70 +443,65 @@ public class Overseer implements SolrCloseable {
zkController.getNodeName());
this.id = id;
- this.context = (OverseerElectionContext) context;
+//
+// stateManagmentExecutor = ParWork.getParExecutorService("stateManagmentExecutor",
+// 1, 1, 3000, new SynchronousQueue());
+// taskExecutor = ParWork.getParExecutorService("overseerTaskExecutor",
+// 4, 16, 3000, new SynchronousQueue());
// try {
// if (context != null) context.close();
// } catch (Exception e) {
// log.error("", e);
// }
- Http2SolrClient.Builder overseerOnlyClientBuilder = new Http2SolrClient.Builder();
- overseerOnlyClientBuilder = overseerOnlyClientBuilder.connectionTimeout(15000).idleTimeout(500000);
-
+ if (overseerOnlyClient == null && !closeAndDone && !initedHttpClient) {
+ overseerOnlyClient = new Http2SolrClient.Builder().idleTimeout(500000).markInternalRequest().build();
+ overseerOnlyClient.enableCloseLock();
+ this.overseerLbClient = new LBHttp2SolrClient(overseerOnlyClient);
+ initedHttpClient = true;
+ }
- overseerOnlyClient = overseerOnlyClientBuilder.markInternalRequest().build();
- overseerOnlyClient.enableCloseLock();
- this.overseerLbClient = new LBHttp2SolrClient(overseerOnlyClient);
- try {
- if (log.isDebugEnabled()) {
- log.debug("set watch on leader znode");
- }
- zkController.getZkClient().exists(Overseer.OVERSEER_ELECT + "/leader", new Watcher() {
+// try {
+// if (log.isDebugEnabled()) {
+// log.debug("set watch on leader znode");
+// }
+// zkController.getZkClient().exists(Overseer.OVERSEER_ELECT + "/leader", new Watcher() {
+//
+// @Override
+// public void process(WatchedEvent event) {
+// if (Event.EventType.None.equals(event.getType())) {
+// return;
+// }
+// if (!isClosed()) {
+// log.info("Overseer leader has changed, closing ...");
+// Overseer.this.close();
+// }
+// }}, true);
+// } catch (KeeperException.SessionExpiredException e) {
+// log.warn("ZooKeeper session expired");
+// return;
+// } catch (InterruptedException | AlreadyClosedException e) {
+// log.info("Already closed");
+// return;
+// } catch (Exception e) {
+// log.error("Unexpected error in Overseer state update loop", e);
+// throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+// }
- @Override
- public void process(WatchedEvent event) {
- if (Event.EventType.None.equals(event.getType())) {
- return;
- }
- if (!isClosed()) {
- log.info("Overseer leader has changed, closing ...");
- Overseer.this.close();
- }
- }}, true);
- } catch (KeeperException.SessionExpiredException e) {
- log.warn("ZooKeeper session expired");
- return;
- } catch (InterruptedException | AlreadyClosedException e) {
- log.info("Already closed");
- return;
- } catch (Exception e) {
- log.error("Unexpected error in Overseer state update loop", e);
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- }
stats = new Stats();
log.info("Overseer (id={}) starting", id);
//launch cluster state updater thread
- ThreadGroup tg = new ThreadGroup("Overseer state updater.");
- updaterThread = new OverseerThread(tg, new ClusterStateUpdater(reader, id, stats), "OverseerStateUpdate-" + id);
- updaterThread.setDaemon(true);
ThreadGroup ccTg = new ThreadGroup("Overseer collection creation process.");
- // nocommit - I don't know about this guy..
- OverseerNodePrioritizer overseerPrioritizer = null; // new OverseerNodePrioritizer(reader, getStateUpdateQueue(), adminPath, shardHandler.getShardHandlerFactory(), updateShardHandler.getUpdateOnlyHttpClient());
- overseerCollectionConfigSetProcessor = new OverseerCollectionConfigSetProcessor(zkController.getCoreContainer(), reader, id, overseerLbClient, adminPath, stats, Overseer.this, overseerPrioritizer);
- ccThread = new OverseerThread(ccTg, overseerCollectionConfigSetProcessor, "OverseerCollectionConfigSetProcessor-" + id);
- ccThread.setDaemon(true);
- updaterThread.start();
- ccThread.start();
- if (triggerThread != null) {
- triggerThread.start();
- }
+ this.zkStateWriter = new ZkStateWriter(reader, stats);
+ //systemCollectionCompatCheck(new StringBiConsumer());
- systemCollectionCompatCheck(new StringBiConsumer());
+ queueWatcher = new WorkQueueWatcher(getCoreContainer());
+ collectionQueueWatcher = new CollectionWorkQueueWatcher(getCoreContainer(), id, overseerLbClient, adminPath, stats, Overseer.this);
// TODO: don't track for a moment, can leak out of collection api tests
// assert ObjectReleaseTracker.track(this);
@@ -816,46 +630,41 @@ public class Overseer implements SolrCloseable {
return zkController.getSolrCloudManager();
}
- /**
- * For tests.
- *
- * @lucene.internal
- * @return state updater thread
- */
- public synchronized OverseerThread getUpdaterThread() {
- return updaterThread;
- }
-
- /**
- * For tests.
- * @lucene.internal
- * @return trigger thread
- */
- public synchronized OverseerThread getTriggerThread() {
- return triggerThread;
- }
-
public void closeAndDone() {
this.closeAndDone = true;
this.closed = true;
}
+ public boolean isCloseAndDone() {
+ return closeAndDone;
+ }
+
public void close() {
close(false);
}
public void close(boolean fromCSUpdateThread) {
- if (this.id != null) {
- log.info("Overseer (id={}) closing", id);
- }
- if (context != null) {
- try {
- context.close(fromCSUpdateThread);
- } finally {
- if (!closeAndDone) {
- zkController.rejoinOverseerElection(context.electionPath, false);
- }
+ log.info("Overseer (id={}) closing closeAndDone={} frp,CSUpdateThread={}", id, closeAndDone, fromCSUpdateThread);
+
+ if (closeAndDone) {
+ if (overseerOnlyClient != null) {
+ overseerOnlyClient.disableCloseLock();
+ }
+
+ if (overseerLbClient != null) {
+ overseerLbClient.close();
+ overseerLbClient = null;
+ }
+
+ if (overseerOnlyClient != null) {
+ overseerOnlyClient.close();
+ overseerOnlyClient = null;
+ }
+ } else {
+ if (!zkController.getCoreContainer().isShutDown() && !zkController.isShudownCalled() && !zkController.isClosed()) {
+ log.info("rejoining the overseer election after closing");
+ zkController.rejoinOverseerElection( false);
}
}
}
@@ -871,59 +680,41 @@ public class Overseer implements SolrCloseable {
void doClose(boolean fromCSUpdateThread) {
closed = true;
- if (log.isDebugEnabled()) {
- log.debug("doClose() - start");
- }
- if (ccThread != null) {
- ((OverseerCollectionConfigSetProcessor) ccThread.getThread()).closing();
- }
+ log.info("doClose() - start fromCSUpdateThread={} closeAndDone={}", fromCSUpdateThread, closeAndDone);
+ this.zkStateWriter = null;
- if (overseerLbClient != null) {
- overseerLbClient.close();
- overseerLbClient = null;
+ if (queueWatcher != null) {
+ queueWatcher.close();
}
- if (overseerOnlyClient != null) {
- overseerOnlyClient.disableCloseLock();
- overseerOnlyClient.close();
- overseerLbClient = null;
+ if (collectionQueueWatcher != null) {
+ collectionQueueWatcher.close();
}
- if (ccThread != null) {
- ccThread.interrupt();
- ((OverseerCollectionConfigSetProcessor) ccThread.getThread()).close(closeAndDone);
- }
- if (updaterThread != null) {
- updaterThread.interrupt();
- try {
- updaterThread.getThread().close();
- } catch (Exception e) {
- log.warn("", e);
- }
- }
+// if (stateManagmentExecutor != null) {
+// log.info("shutdown stateManagmentExecutor");
+// stateManagmentExecutor.shutdown();
+// }
+//
+// if (taskExecutor != null) {
+// log.info("shutdown taskExecutor");
+// taskExecutor.shutdown();
+// }
- if (ccThread != null) {
- while (true) {
- try {
- ccThread.join();
- break;
- } catch (InterruptedException e) {
- // okay
- }
- }
- }
- if (updaterThread != null && !fromCSUpdateThread) {
- while (true) {
- try {
- updaterThread.join();
- break;
- } catch (InterruptedException e) {
- // okay
- }
- }
- }
+// if (stateManagmentExecutor != null) {
+// stateManagmentExecutor.shutdownNow();
+// }
+
+ // ExecutorUtil.shutdownAndAwaitTermination(stateManagmentExecutor );
+
+
+// if (taskExecutor != null) {
+// taskExecutor.shutdownNow();
+// }
+
+ // ExecutorUtil.shutdownAndAwaitTermination(taskExecutor );
if (log.isDebugEnabled()) {
log.debug("doClose() - end");
@@ -962,7 +753,7 @@ public class Overseer implements SolrCloseable {
ZkDistributedQueue getStateUpdateQueue(Stats zkStats) {
return new ZkDistributedQueue(reader.getZkClient(), "/overseer/queue", zkStats, STATE_UPDATE_MAX_QUEUE, new ConnectionManager.IsClosed(){
public boolean isClosed() {
- return Overseer.this.isClosed() || zkController.getCoreContainer().isShutDown();
+ return Overseer.this.isClosed() || zkController.getCoreContainer().isShutDown(); // nocommit use
}
});
}
@@ -1076,8 +867,326 @@ public class Overseer implements SolrCloseable {
return reader;
}
+ public ZkStateWriter getZkStateWriter() {
+ return zkStateWriter;
+ }
+
public void offerStateUpdate(byte[] data) throws KeeperException, InterruptedException {
getStateUpdateQueue().offer(data);
}
+ public boolean processQueueItem(ZkNodeProps message) throws InterruptedException {
+ log.info("processQueueItem {}", message);
+ // nocommit - may not need this now
+ new OverseerTaskExecutorTask(getCoreContainer(), zkStateWriter, message).run();
+// try {
+// future.get();
+// } catch (ExecutionException e) {
+// log.error("", e);
+// throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+// }
+ return true;
+ }
+
+ public void writePendingUpdates() throws InterruptedException {
+ log.info("writePendingUpdates ");
+
+ new OverseerTaskExecutorTask.WriteTask(getCoreContainer(), zkStateWriter).run();
+// try {
+// future.get();
+// } catch (ExecutionException e) {
+// log.error("", e);
+// throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+// }
+
+ }
+
+ // nocommit use
+ private static class OverseerWatcher implements Watcher, Closeable {
+
+ private final CoreContainer cc;
+ private final ZkController zkController;
+ private final String path = Overseer.OVERSEER_ELECT + "/leader";
+ private final Overseer overseer;
+
+ public OverseerWatcher(CoreContainer cc) {
+ this.cc = cc;
+ this.zkController = cc.getZkController();
+ this.overseer = zkController.getOverseer();
+ }
+
+ @Override
+ public void process(WatchedEvent event) {
+ try {
+ if (log.isDebugEnabled()) {
+ log.debug("set watch on leader znode");
+ }
+ zkController.getZkClient().exists(path, new Watcher() {
+
+ @Override
+ public void process(WatchedEvent event) {
+ if (Event.EventType.None.equals(event.getType())) {
+ return;
+ }
+
+ log.info("Overseer leader has changed, closing ...");
+ overseer.close();
+
+ }}, true);
+ } catch (KeeperException.SessionExpiredException e) {
+ log.warn("ZooKeeper session expired");
+ overseer.doClose(false);
+ return;
+ } catch (InterruptedException | AlreadyClosedException e) {
+ log.info("Already closed");
+ overseer.doClose(false);
+ return;
+ } catch (Exception e) {
+ log.error("Unexpected error in Overseer state update loop", e);
+ overseer.doClose(false);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ zkController.getZkClient().getSolrZooKeeper().removeWatches(path, this, WatcherType.Data, true);
+ } catch (Exception e) {
+ log.info("", e.getMessage());
+ }
+ }
+ }
+
+ private static abstract class QueueWatcher implements Watcher, Closeable {
+
+ protected final CoreContainer cc;
+ protected final ZkController zkController;
+ protected final String path;
+ protected final Overseer overseer;
+ protected volatile boolean closed;
+
+ public QueueWatcher(CoreContainer cc, String path) throws KeeperException {
+ this.cc = cc;
+ this.zkController = cc.getZkController();
+ this.overseer = zkController.getOverseer();
+ this.path = path;
+ List<String> items = setWatch();
+ log.info("Overseer found entries on start {}", items);
+ processQueueItems(items);
+ }
+
+ private List<String> setWatch() {
+ try {
+
+ log.info("set watch on Overseer work queue {}", path);
+
+ List<String> children = zkController.getZkClient().getChildren(path, this, true);
+ Collections.sort(children);
+ return children;
+ } catch (KeeperException.SessionExpiredException e) {
+ log.warn("ZooKeeper session expired");
+ overseer.doClose(false);
+ return null;
+ } catch (InterruptedException | AlreadyClosedException e) {
+ log.info("Already closed");
+ overseer.doClose(false);
+ return null;
+ } catch (Exception e) {
+ log.error("Unexpected error in Overseer state update loop", e);
+ overseer.doClose(false);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+ }
+
+ @Override
+ public synchronized void process(WatchedEvent event) {
+ if (Event.EventType.None.equals(event.getType())) {
+ return;
+ }
+ if (this.closed) {
+ return;
+ }
+
+ log.info("Overseer work queue has changed, processing...");
+
+ try {
+ List<String> items = setWatch();
+
+ processQueueItems(items);
+ } catch (Exception e) {
+ log.error("Exception during overseer queue queue processing", e);
+ }
+
+ }
+
+ protected abstract void processQueueItems(List<String> items) throws KeeperException;
+
+ @Override
+ public void close() {
+ this.closed = true;
+ try {
+ zkController.getZkClient().getSolrZooKeeper().removeWatches(path, this, WatcherType.Data, true);
+ } catch (Exception e) {
+ log.info("", e.getMessage());
+ }
+ }
+ }
+
+ private static class WorkQueueWatcher extends QueueWatcher {
+
+ public WorkQueueWatcher(CoreContainer cc) throws KeeperException {
+ super(cc, Overseer.OVERSEER_QUEUE);
+ }
+
+ @Override
+ protected void processQueueItems(List<String> items) {
+ log.info("Found state update queue items {}", items);
+ List<String> fullPaths = new ArrayList<>(items.size());
+ for (String item : items) {
+ fullPaths.add(path + "/" + item);
+ }
+
+ Map<String,byte[]> data = zkController.getZkClient().getData(fullPaths);
+
+ for (byte[] item : data.values()) {
+ final ZkNodeProps message = ZkNodeProps.load(item);
+ try {
+ boolean success = overseer.processQueueItem(message);
+ } catch (InterruptedException e) {
+ log.error("Overseer state update queue processing interrupted");
+ return;
+ }
+ }
+
+ try {
+ overseer.writePendingUpdates();
+ } catch (InterruptedException e) {
+ log.error("Overseer state update queue processing interrupted");
+ return;
+ }
+
+ zkController.getZkClient().delete(fullPaths, true);
+ }
+ }
+
+ private static class CollectionWorkQueueWatcher extends QueueWatcher {
+
+ private final OverseerCollectionMessageHandler collMessageHandler;
+ private final OverseerConfigSetMessageHandler configMessageHandler;
+ private final DistributedMap failureMap;
+ private final DistributedMap runningMap;
+
+ private final DistributedMap completedMap;
+
+ public CollectionWorkQueueWatcher(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer) throws KeeperException {
+ super(cc, Overseer.OVERSEER_COLLECTION_QUEUE_WORK);
+ collMessageHandler = new OverseerCollectionMessageHandler(cc, myId, overseerLbClient, adminPath, stats, overseer);
+ configMessageHandler = new OverseerConfigSetMessageHandler(cc);
+ failureMap = Overseer.getFailureMap(cc.getZkController().getZkStateReader().getZkClient());
+ runningMap = Overseer.getRunningMap(cc.getZkController().getZkStateReader().getZkClient());
+ completedMap = Overseer.getCompletedMap(cc.getZkController().getZkStateReader().getZkClient());
+ }
+
+ @Override
+ public void close() {
+ super.close();
+ IOUtils.closeQuietly(collMessageHandler);
+ IOUtils.closeQuietly(configMessageHandler);
+ }
+
+ @Override
+ protected synchronized void processQueueItems(List<String> items) throws KeeperException {
+ log.info("Found collection queue items {}", items);
+ List<String> fullPaths = new ArrayList<>(items.size());
+ for (String item : items) {
+ fullPaths.add(path + "/" + item);
+ }
+
+ Map<String,byte[]> data = zkController.getZkClient().getData(fullPaths);
+
+ ParWork.getRootSharedExecutor().submit(()->{
+ try {
+ runAsync(items, fullPaths, data);
+ } catch (Exception e) {
+ log.error("failed processing collection queue items " + items);
+ }
+ });
+
+ }
+
+ private void runAsync(List<String> items, List<String> fullPaths, Map<String,byte[]> data) throws KeeperException {
+ for (Map.Entry<String,byte[]> entry : data.entrySet()) {
+ byte[] item = entry.getValue();
+ if (item == null) {
+ log.error("empty item {}", entry.getKey());
+ continue;
+ }
+
+ final ZkNodeProps message = ZkNodeProps.load(item);
+ try {
+ String operation = message.getStr(Overseer.QUEUE_OPERATION);
+ if (operation == null) {
+ log.error("Msg does not have required " + Overseer.QUEUE_OPERATION + ": {}", message);
+ continue;
+ }
+
+ final String asyncId = message.getStr(ASYNC);
+
+ OverseerSolrResponse response;
+ if (operation != null && operation.startsWith(CONFIGSETS_ACTION_PREFIX)) {
+ response = configMessageHandler.processMessage(message, operation);
+ } else {
+ response = collMessageHandler.processMessage(message, operation);
+ }
+
+
+// try {
+// overseer.writePendingUpdates();
+// } catch (InterruptedException e) {
+// log.error("Overseer state update queue processing interrupted");
+// return;
+// }
+
+ log.info("response {}", response);
+
+
+ if (asyncId != null) {
+ if (response != null && (response.getResponse().get("failure") != null || response.getResponse().get("exception") != null)) {
+ if (log.isDebugEnabled()) {
+ log.debug("Updated failed map for task with id:[{}]", asyncId);
+ }
+ failureMap.put(asyncId, OverseerSolrResponseSerializer.serialize(response));
+ } else {
+ if (log.isDebugEnabled()) {
+ log.debug("Updated completed map for task with zkid:[{}]", asyncId);
+ }
+ completedMap.put(asyncId, OverseerSolrResponseSerializer.serialize(response));
+
+ }
+ } else {
+ byte[] sdata = OverseerSolrResponseSerializer.serialize(response);
+ String responsePath = Overseer.OVERSEER_COLLECTION_MAP_COMPLETED + "/" + OverseerTaskQueue.RESPONSE_PREFIX
+ + entry.getKey().substring(entry.getKey().lastIndexOf("-") + 1);
+ zkController.getZkClient().setData( responsePath, sdata, true);
+ log.debug("Completed task:[{}] {}", message, response.getResponse());
+ }
+
+
+ } catch (InterruptedException e) {
+ log.error("Overseer state update queue processing interrupted");
+ return;
+ }
+ }
+
+ for (String item : items) {
+ if (item.startsWith("qnr-")) {
+ fullPaths.remove(path + "/" + item);
+ }
+ }
+
+ zkController.getZkClient().delete(fullPaths, true);
+ }
+ }
+
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java
index 8b2e1c7..ea3ef9a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java
@@ -42,65 +42,24 @@ import org.apache.zookeeper.KeeperException;
*/
public class OverseerCollectionConfigSetProcessor extends OverseerTaskProcessor {
- public OverseerCollectionConfigSetProcessor(CoreContainer cc, ZkStateReader zkStateReader, String myId, LBHttp2SolrClient overseerLbClient,
- String adminPath, Stats stats, Overseer overseer,
- OverseerNodePrioritizer overseerNodePrioritizer) throws KeeperException {
- this(cc,
- zkStateReader,
- myId,
- adminPath,
- stats,
- overseerLbClient, (HttpShardHandlerFactory) cc.getShardHandlerFactory(),
- overseer,
- overseerNodePrioritizer,
- overseer.getCollectionQueue(zkStateReader.getZkClient(), stats),
- Overseer.getRunningMap(zkStateReader.getZkClient()),
- Overseer.getCompletedMap(zkStateReader.getZkClient()),
- Overseer.getFailureMap(zkStateReader.getZkClient())
- );
+ public OverseerCollectionConfigSetProcessor(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer) throws KeeperException {
+ this(cc, myId, overseerLbClient, adminPath, stats, overseer, overseer.getCollectionQueue(cc.getZkController().getZkStateReader().getZkClient(), stats),
+ Overseer.getRunningMap(cc.getZkController().getZkStateReader().getZkClient()), Overseer.getCompletedMap(cc.getZkController().getZkStateReader().getZkClient()),
+ Overseer.getFailureMap(cc.getZkController().getZkStateReader().getZkClient()));
}
- protected OverseerCollectionConfigSetProcessor(CoreContainer cc, ZkStateReader zkStateReader, String myId,
- String adminPath,
- Stats stats,
- LBHttp2SolrClient overseerLbClient,
- HttpShardHandlerFactory shardHandlerFactory,
- Overseer overseer,
- OverseerNodePrioritizer overseerNodePrioritizer,
- OverseerTaskQueue workQueue,
- DistributedMap runningMap,
- DistributedMap completedMap,
- DistributedMap failureMap) {
- super(
- cc,
- myId,
- stats,
- getOverseerMessageHandlerSelector(zkStateReader, myId, overseerLbClient, shardHandlerFactory,
- adminPath, stats, overseer, overseerNodePrioritizer),
- overseerNodePrioritizer,
- workQueue,
- runningMap,
- completedMap,
- failureMap);
+ protected OverseerCollectionConfigSetProcessor(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer, OverseerTaskQueue workQueue,
+ DistributedMap runningMap, DistributedMap completedMap, DistributedMap failureMap) {
+ super(cc, myId, stats, getOverseerMessageHandlerSelector(cc, myId, overseerLbClient, adminPath, stats, overseer), workQueue, runningMap, completedMap, failureMap);
}
- private static OverseerMessageHandlerSelector getOverseerMessageHandlerSelector(
- ZkStateReader zkStateReader,
- String myId,
- LBHttp2SolrClient overseerLbClient,
- HttpShardHandlerFactory shardHandlerFactory,
- String adminPath,
- Stats stats,
- Overseer overseer,
- OverseerNodePrioritizer overseerNodePrioritizer) {
- final OverseerCollectionMessageHandler collMessageHandler = new OverseerCollectionMessageHandler(
- zkStateReader, myId, overseerLbClient, shardHandlerFactory, adminPath, stats, overseer, overseerNodePrioritizer);
- final OverseerConfigSetMessageHandler configMessageHandler = new OverseerConfigSetMessageHandler(
- zkStateReader);
+ private static OverseerMessageHandlerSelector getOverseerMessageHandlerSelector(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer) {
+
+ final OverseerConfigSetMessageHandler configMessageHandler = new OverseerConfigSetMessageHandler(cc);
return new OverseerMessageHandlerSelector() {
@Override
public void close() throws IOException {
- IOUtils.closeQuietly(collMessageHandler);
+
IOUtils.closeQuietly(configMessageHandler);
}
@@ -110,7 +69,7 @@ public class OverseerCollectionConfigSetProcessor extends OverseerTaskProcessor
if (operation != null && operation.startsWith(CONFIGSETS_ACTION_PREFIX)) {
return configMessageHandler;
}
- return collMessageHandler;
+ throw new IllegalArgumentException("No handler for " + operation + " " + message);
}
};
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
index ab95e2a..c70048d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.params.ConfigSetParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.ConfigSetProperties;
+import org.apache.solr.core.CoreContainer;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
@@ -83,8 +84,8 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- public OverseerConfigSetMessageHandler(ZkStateReader zkStateReader) {
- this.zkStateReader = zkStateReader;
+ public OverseerConfigSetMessageHandler(CoreContainer cc) {
+ this.zkStateReader = cc.getZkController().getZkStateReader();
this.configSetWriteWip = new HashSet();
this.configSetReadWip = new HashSet();
}
@@ -117,9 +118,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
+ operation);
}
} catch (Exception e) {
- // interrupt not currently thrown here, but it could be - I
- // usually like to use a utility everywhere for this reason
- ParWork.propagateInterrupt(e);
+ log.error("Operation " + operation + " caused exception", e);
String configSetName = message.getStr(NAME);
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
index 6913b48..71fa2d3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
@@ -21,10 +21,14 @@ import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.cloud.ConnectionManager;
+import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.util.Pair;
@@ -41,15 +45,24 @@ final class OverseerElectionContext extends ShardLeaderElectionContextBase {
private volatile boolean isClosed = false;
public OverseerElectionContext(final String zkNodeName, SolrZkClient zkClient, Overseer overseer) {
- super(zkNodeName, Overseer.OVERSEER_ELECT, Overseer.OVERSEER_ELECT + "/leader", new ZkNodeProps(ID, zkNodeName), zkClient);
+ super(zkNodeName, Overseer.OVERSEER_ELECT, Overseer.OVERSEER_ELECT + "/leader", new Replica(ID, getIDMap(zkNodeName), null, null), zkClient);
this.overseer = overseer;
this.zkClient = zkClient;
}
+ private static Map<String,Object> getIDMap(String zkNodeName) {
+ Map<String,Object> idMap = new HashMap<>(1);
+ idMap.put(ID, zkNodeName);
+ return idMap;
+ }
+
@Override
void runLeaderProcess(ElectionContext context, boolean weAreReplacement, int pauseBeforeStartMs) throws KeeperException,
InterruptedException, IOException {
- if (isClosed() || overseer.isDone()) {
+ log.info("Running the leader process for Overseer");
+
+ if (overseer.isDone()) {
+ log.info("Already closed, bailing ...");
return;
}
@@ -70,37 +83,22 @@ final class OverseerElectionContext extends ShardLeaderElectionContextBase {
// clearQueue(Overseer.getInternalWorkQueue(zkClient, new Stats()));
// }
- log.info("Running the leader process for Overseer");
super.runLeaderProcess(context, weAreReplacement, pauseBeforeStartMs);
log.info("Registered as Overseer leader, starting Overseer ...");
- if (isClosed()) {
- log.info("Bailing on becoming leader, we are closed");
- return;
- }
- if (!isClosed() && !overseer.getZkController().getCoreContainer().isShutDown() && !overseer.isDone() && (overseer.getUpdaterThread() == null || !overseer.getUpdaterThread().isAlive())) {
+ if (!overseer.getZkController().getCoreContainer().isShutDown() && !overseer.getZkController().isShudownCalled()
+ && !overseer.isDone()) {
+ log.info("Starting overseer after winnning Overseer election {}", id);
overseer.start(id, context);
+ } else {
+ log.info("Will not start Overseer because we are closed");
+ cancelElection();
}
}
- private void clearQueue(ZkDistributedQueue queue)
- throws KeeperException, InterruptedException {
- while (true) {
- Collection<Pair<String,byte[]>> items = queue.peekElements(1000, 0, null);
- List<String> paths = new ArrayList<>(items.size());
- if (items.size() == 0) {
- break;
- }
- for (Pair<String, byte[]> item : items) {
- paths.add(item.first());
- }
- queue.remove(paths);
- }
- }
-
public Overseer getOverseer() {
return overseer;
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskExecutorTask.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskExecutorTask.java
new file mode 100644
index 0000000..90d5079
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskExecutorTask.java
@@ -0,0 +1,157 @@
+package org.apache.solr.cloud;
+
+import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.NodeMutator;
+import org.apache.solr.cloud.overseer.OverseerAction;
+import org.apache.solr.cloud.overseer.ReplicaMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
+import org.apache.solr.cloud.overseer.ZkStateWriter;
+import org.apache.solr.cloud.overseer.ZkWriteCommand;
+import org.apache.solr.common.ParWork;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.core.CoreContainer;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public class OverseerTaskExecutorTask implements Runnable {
+ private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+ private final ZkController zkController;
+ private final SolrCloudManager cloudManager;
+ private final SolrZkClient zkClient;
+ private final Overseer overseer;
+ private final ZkStateWriter zkStateWriter;
+ private final ZkNodeProps message;
+
+ public OverseerTaskExecutorTask(CoreContainer cc, ZkStateWriter zkStateWriter, ZkNodeProps message) {
+ this.zkController = cc.getZkController();
+ this.zkClient = zkController.getZkClient();
+ this.cloudManager = zkController.getSolrCloudManager();
+ this.overseer = zkController.getOverseer();
+ this.zkStateWriter = zkStateWriter;
+ this.message = message;
+ }
+
+
+ private void processQueueItem(ZkNodeProps message) throws Exception {
+ if (log.isDebugEnabled()) log.debug("Consume state update from queue {} {}", message);
+
+ // assert clusterState != null;
+
+ // if (clusterState.getZNodeVersion() == 0 || clusterState.getZNodeVersion() > lastVersion) {
+
+ final String operation = message.getStr(Overseer.QUEUE_OPERATION);
+ if (operation == null) {
+ log.error("Message missing " + Overseer.QUEUE_OPERATION + ":" + message);
+ return;
+ }
+
+ log.info("Queue operation is {}", operation);
+
+ ClusterState cs = zkStateWriter.getClusterstate(true);
+
+ log.info("Process message {} {}", message, operation);
+ ClusterState newClusterState = processMessage(message, operation, cs);
+
+ log.info("Enqueue message {}", operation);
+ zkStateWriter.enqueueUpdate(newClusterState, true);
+
+
+ if (log.isDebugEnabled()) log.debug("State update consumed from queue {}", message);
+ }
+
+ private ClusterState processMessage(final ZkNodeProps message, final String operation, ClusterState clusterState) {
+ if (log.isDebugEnabled()) {
+ log.debug("processMessage(ZkNodeProps message={}, String operation={} clusterState={})", message, operation, clusterState);
+ }
+
+ OverseerAction overseerAction = OverseerAction.get(operation);
+ if (overseerAction == null) {
+ throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
+ }
+ switch (overseerAction) {
+ case STATE:
+ return new ReplicaMutator(cloudManager).setState(clusterState, message);
+ case LEADER:
+ return new SliceMutator(cloudManager).setShardLeader(clusterState, message);
+ case ADDROUTINGRULE:
+ return new SliceMutator(cloudManager).addRoutingRule(clusterState, message);
+ case REMOVEROUTINGRULE:
+ return new SliceMutator(cloudManager).removeRoutingRule(clusterState, message);
+ case UPDATESHARDSTATE:
+ return new SliceMutator(cloudManager).updateShardState(clusterState, message);
+ // case QUIT:
+ // if (myId.equals(message.get(ID))) {
+ // log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId));
+ // try {
+ // overseerCollectionConfigSetProcessor.close();
+ // } catch (IOException e) {
+ // log.error("IOException", e);
+ // }
+ // close();
+ // } else {
+ // log.warn("Overseer received wrong QUIT message {}", message);
+ // }
+ // break;
+ case DOWNNODE:
+ return new NodeMutator().downNode(clusterState, message);
+ default:
+ throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
+
+ }
+ }
+
+ @Override
+ public void run() {
+ log.info("OverseerTaskExecutorTask, going to process message {}", message);
+
+ try {
+ processQueueItem(message);
+ } catch (Exception e) {
+ log.error("Failed to process message " + message, e);
+ }
+ }
+
+ public static class WriteTask implements Runnable {
+ CoreContainer coreContainer;
+ ZkStateWriter zkStateWriter;
+
+ public WriteTask(CoreContainer coreContainer, ZkStateWriter zkStateWriter) {
+ this.coreContainer = coreContainer;
+ this.zkStateWriter = zkStateWriter;
+ }
+
+ @Override
+ public void run() {
+ try {
+ zkStateWriter.writePendingUpdates();
+ } catch (NullPointerException e) {
+ if (log.isDebugEnabled()) log.debug("Won't write pending updates, zkStateWriter=null");
+ } catch (Exception e) {
+ log.error("Failed to process pending updates", e);
+ }
+ }
+ }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
index 0b6498d..628a32e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
@@ -125,15 +125,12 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
protected final OverseerMessageHandlerSelector selector;
- private final OverseerNodePrioritizer prioritizer;
-
private final String thisNode;
private Map<Runner,Future> taskFutures = new ConcurrentHashMap<>();
public OverseerTaskProcessor(CoreContainer cc, String myId,
Stats stats,
OverseerMessageHandlerSelector selector,
- OverseerNodePrioritizer prioritizer,
OverseerTaskQueue workQueue,
DistributedMap runningMap,
DistributedMap completedMap,
@@ -141,7 +138,6 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
this.myId = myId;
this.stats = stats;
this.selector = selector;
- this.prioritizer = prioritizer;
this.workQueue = workQueue;
this.runningMap = runningMap;
this.completedMap = completedMap;
@@ -350,9 +346,9 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
if (closeAndDone) {
// nocommit
-// for (Future future : taskFutures.values()) {
-// future.cancel(false);
-// }
+ // for (Future future : taskFutures.values()) {
+ // future.cancel(false);
+ // }
for (Future future : taskFutures.values()) {
try {
future.get(1, TimeUnit.SECONDS);
@@ -378,8 +374,8 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
public static List<String> getSortedElectionNodes(SolrZkClient zk, String path) throws KeeperException, InterruptedException {
List<String> children = zk.getChildren(path, null, true);
- LeaderElector.sortSeqs(children);
- return children;
+ LeaderElector.sortSeqs(children);
+ return children;
}
public static String getLeaderNode(SolrZkClient zkClient) throws KeeperException, InterruptedException {
@@ -533,15 +529,15 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
private void resetTaskWithException(String id, String asyncId, String taskKey) throws KeeperException, InterruptedException {
log.warn("Resetting task: {}, requestid: {}, taskKey: {}", id, asyncId, taskKey);
- if (asyncId != null) {
- if (!runningMap.remove(asyncId)) {
- log.warn("Could not find and remove async call [{}] from the running map.", asyncId);
- }
+ if (asyncId != null) {
+ if (!runningMap.remove(asyncId)) {
+ log.warn("Could not find and remove async call [{}] from the running map.", asyncId);
}
+ }
- synchronized (runningTasks) {
- runningTasks.remove(id);
- }
+ synchronized (runningTasks) {
+ runningTasks.remove(id);
+ }
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
index eabc34f..9c0f6f9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
@@ -16,31 +16,29 @@
*/
package org.apache.solr.cloud;
-import java.io.File;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.LongAdder;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Predicate;
import com.codahale.metrics.Timer;
-import org.apache.solr.common.ParWork;
+import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -52,9 +50,11 @@ import org.slf4j.LoggerFactory;
public class OverseerTaskQueue extends ZkDistributedQueue {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- public static final String RESPONSE_PREFIX = "qnr-" ;
+ static final String RESPONSE_PREFIX = "qnr-" ;
+ public static final byte[] BYTES = new byte[0];
- private final LongAdder pendingResponses = new LongAdder();
+ private final AtomicBoolean shuttingDown = new AtomicBoolean(false);
+ private final AtomicInteger pendingResponses = new AtomicInteger(0);
public OverseerTaskQueue(SolrZkClient zookeeper, String dir) {
this(zookeeper, dir, new Stats());
@@ -65,11 +65,11 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
}
public void allowOverseerPendingTasksToComplete() {
- while (pendingResponses.sum() > 0) {
+ shuttingDown.set(true);
+ while (pendingResponses.get() > 0) {
try {
- Thread.sleep(100);
+ Thread.sleep(50);
} catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
log.error("Interrupted while waiting for overseer queue to drain before shutdown!");
}
}
@@ -80,47 +80,28 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
*/
public boolean containsTaskWithRequestId(String requestIdKey, String requestId)
throws KeeperException, InterruptedException {
- Set<String> childNames;
- updateLock.lockInterruptibly();
- try {
-
- childNames = knownChildren.keySet();
-
- stats.setQueueLength(childNames.size());
- for (String childName : childNames) {
- if (childName != null && childName.startsWith(PREFIX)) {
- try {
- byte[] data = knownChildren.get(childName);
- if (data == null) {
- data = zookeeper.getData(dir + "/" + childName, null, null, true);
- if (data != null) {
- knownChildren.put(childName, data);
- }
- }
- if (data != null) {
- ZkNodeProps message = ZkNodeProps.load(data);
- if (message.containsKey(requestIdKey)) {
- if (log.isDebugEnabled()) {
- log.debug("Looking for {}, found {}", message.get(requestIdKey), requestId);
- }
- if(message.get(requestIdKey).equals(requestId)) return true;
+ List<String> childNames = zookeeper.getChildren(dir, null, true);
+ stats.setQueueLength(childNames.size());
+ for (String childName : childNames) {
+ if (childName != null && childName.startsWith(PREFIX)) {
+ try {
+ byte[] data = zookeeper.getData(dir + "/" + childName, null, null, true);
+ if (data != null) {
+ ZkNodeProps message = ZkNodeProps.load(data);
+ if (message.containsKey(requestIdKey)) {
+ if (log.isDebugEnabled()) {
+ log.debug("Looking for {}, found {}", message.get(requestIdKey), requestId);
}
+ if(message.get(requestIdKey).equals(requestId)) return true;
}
- } catch (KeeperException.NoNodeException e) {
- knownChildren.remove(childName);
}
+ } catch (KeeperException.NoNodeException e) {
+ // Another client removed the node first, try next
}
}
-
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
}
-
-
return false;
}
@@ -131,30 +112,20 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
InterruptedException {
Timer.Context time = stats.time(dir + "_remove_event");
try {
- String path = dir + "/" + event.getId();
- String responseId = RESPONSE_PREFIX
+ String path = event.getId();
+ String responsePath = dir + "/" + RESPONSE_PREFIX
+ path.substring(path.lastIndexOf("-") + 1);
- String responsePath = dir + "/" + responseId;
try {
zookeeper.setData(responsePath, event.getBytes(), true);
} catch (KeeperException.NoNodeException ignored) {
- // this will often not exist or have been removed - nocommit - debug this response stuff
- if (log.isDebugEnabled()) log.debug("Response ZK path: {} doesn't exist.", responsePath);
+ // we must handle the race case where the node no longer exists
+ log.info("Response ZK path: {} doesn't exist. Requestor may have disconnected from ZooKeeper", responsePath);
}
try {
zookeeper.delete(path, -1, true);
} catch (KeeperException.NoNodeException ignored) {
}
-
- updateLock.lockInterruptibly();
- try {
- knownChildren.remove(event.getId());
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
} finally {
time.stop();
}
@@ -165,20 +136,16 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
*/
static final class LatchWatcher implements Watcher {
- private final ReentrantLock lock;
+ private final Lock lock;
private final Condition eventReceived;
- private final SolrZkClient zkClient;
- private volatile WatchedEvent event;
- private final Event.EventType latchEventType;
-
- private volatile boolean triggered = false;
+ private WatchedEvent event;
+ private Event.EventType latchEventType;
- LatchWatcher(SolrZkClient zkClient) {
- this(null, zkClient);
+ LatchWatcher() {
+ this(null);
}
- LatchWatcher(Event.EventType eventType, SolrZkClient zkClient) {
- this.zkClient = zkClient;
+ LatchWatcher(Event.EventType eventType) {
this.lock = new ReentrantLock();
this.eventReceived = lock.newCondition();
this.latchEventType = eventType;
@@ -192,47 +159,30 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
return;
}
// If latchEventType is not null, only fire if the type matches
- if (log.isDebugEnabled()) {
- log.debug("{} fired on path {} state {} latchEventType {}", event.getType(), event.getPath(), event.getState(), latchEventType);
- }
+
+ log.info("{} fired on path {} state {} latchEventType {}", event.getType(), event.getPath(), event.getState(), latchEventType);
+
if (latchEventType == null || event.getType() == latchEventType) {
- try {
- lock.lockInterruptibly();
- } catch (InterruptedException e) {
- return;
- }
+ lock.lock();
try {
this.event = event;
- triggered = true;
eventReceived.signalAll();
} finally {
- if (lock.isHeldByCurrentThread()) {
- lock.unlock();
- }
+ lock.unlock();
}
}
}
public void await(long timeoutMs) throws InterruptedException {
assert timeoutMs > 0;
- lock.lockInterruptibly();
+ lock.lock();
try {
if (this.event != null) {
return;
}
- TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
- while (event == null && !timeout.hasTimedOut()) {
- try {
- eventReceived.await(timeoutMs, TimeUnit.MILLISECONDS);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw e;
- }
- }
+ eventReceived.await(timeoutMs, TimeUnit.MILLISECONDS);
} finally {
- if (lock.isHeldByCurrentThread()) {
- lock.unlock();
- }
+ lock.unlock();
}
}
@@ -248,7 +198,17 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
*/
private String createData(String path, byte[] data, CreateMode mode)
throws KeeperException, InterruptedException {
- return zookeeper.create(path, data, mode, true);
+ for (;;) {
+ try {
+ return zookeeper.create(path, data, mode, true);
+ } catch (KeeperException.NoNodeException e) {
+ try {
+ zookeeper.create(dir, BYTES, CreateMode.PERSISTENT, true);
+ } catch (KeeperException.NodeExistsException ne) {
+ // someone created it
+ }
+ }
+ }
}
/**
@@ -257,60 +217,68 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
*/
public QueueEvent offer(byte[] data, long timeout) throws KeeperException,
InterruptedException {
- Timer.Context time = stats.time(dir + "_offer");
+ log.info("offer operation to the Overseeer queue {}", Utils.fromJSON(data));
+ if (shuttingDown.get()) {
+ throw new SolrException(SolrException.ErrorCode.CONFLICT,"Solr is shutting down, no more overseer tasks may be offered");
+ }
+ // Timer.Context time = stats.time(dir + "_offer");
try {
// Create and watch the response node before creating the request node;
// otherwise we may miss the response.
String watchID = createResponseNode();
- LatchWatcher watcher = new LatchWatcher(zookeeper);
- byte[] bytes = zookeeper.getData(watchID, watcher, null, true);
+ log.info("watchId for response node {}, setting a watch ... ", watchID);
- // create the request node
- createRequestNode(data, watchID);
+ LatchWatcher watcher = new LatchWatcher(Watcher.Event.EventType.NodeDataChanged);
+ Stat stat = zookeeper.exists(watchID, watcher, true);
- pendingResponses.increment();
- if (bytes == null) {
+ // create the request node
+ String path = createRequestNode(data, watchID);
+ log.info("created request node at {}", path);
+ if (stat != null) {
+ pendingResponses.incrementAndGet();
+ log.info("wait on latch {}", timeout);
watcher.await(timeout);
- bytes = zookeeper.getData(watchID, null, null, true);
}
-
+ byte[] bytes = zookeeper.getData(watchID, null, null, true);
+ log.info("get data from response node {} {} {}", watchID, bytes == null ? null : bytes.length, watcher.getWatchedEvent());
// create the event before deleting the node, otherwise we can get the deleted
// event from the watcher.
QueueEvent event = new QueueEvent(watchID, bytes, watcher.getWatchedEvent());
+ log.info("delete response node... {}", watchID);
zookeeper.delete(watchID, -1, true);
return event;
} finally {
- time.stop();
- pendingResponses.decrement();
+ // time.stop();
+ pendingResponses.decrementAndGet();
}
}
- void createRequestNode(byte[] data, String watchID) throws KeeperException, InterruptedException {
- createData(dir + "/" + PREFIX + watchID.substring(watchID.lastIndexOf("-") + 1),
+
+ String createRequestNode(byte[] data, String watchID) throws KeeperException, InterruptedException {
+ return createData(dir + "/" + PREFIX + watchID.substring(watchID.lastIndexOf("-") + 1),
data, CreateMode.PERSISTENT);
}
String createResponseNode() throws KeeperException, InterruptedException {
return createData(
- dir + "/" + RESPONSE_PREFIX,
- null, CreateMode.EPHEMERAL_SEQUENTIAL);
+ Overseer.OVERSEER_COLLECTION_MAP_COMPLETED + "/" + RESPONSE_PREFIX,
+ null, CreateMode.EPHEMERAL_SEQUENTIAL);
}
public List<QueueEvent> peekTopN(int n, Predicate<String> excludeSet, long waitMillis)
throws KeeperException, InterruptedException {
- if (log.isDebugEnabled()) log.debug("peekTopN {} {}", n, excludeSet);
ArrayList<QueueEvent> topN = new ArrayList<>();
+ log.debug("Peeking for top {} elements. ExcludeSet: {}", n, excludeSet);
Timer.Context time;
if (waitMillis == Long.MAX_VALUE) time = stats.time(dir + "_peekTopN_wait_forever");
else time = stats.time(dir + "_peekTopN_wait" + waitMillis);
try {
- for (Pair<String, byte[]> element : peekElements(n, waitMillis, excludeSet)) {
- if (log.isDebugEnabled()) log.debug("Add to topN {}", dir + "/" + element.first());
- topN.add(new QueueEvent(new File(element.first()).getName(),
+ for (Pair<String, byte[]> element : peekElements(n, waitMillis, child -> !excludeSet.test(dir + "/" + child))) {
+ topN.add(new QueueEvent(dir + "/" + element.first(),
element.second(), null));
}
printQueueEventsListElementIds(topN);
@@ -337,45 +305,20 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
* Gets last element of the Queue without removing it.
*/
public String getTailId() throws KeeperException, InterruptedException {
+ log.info("getTailId");
// TODO: could we use getChildren here? Unsure what freshness guarantee the caller needs.
- TreeMap<String,byte[]> orderedChildren = fetchZkChildren(null, null);
-
- for (Map.Entry<String,byte[]> headNode : orderedChildren.entrySet())
+ TreeSet<String> orderedChildren = fetchZkChildren(null);
+ log.info("getTailId found {} children", orderedChildren);
+ for (String headNode : orderedChildren.descendingSet())
if (headNode != null) {
try {
- byte[] data;
- updateLock.lockInterruptibly();
- try {
- data = knownChildren.get(headNode.getKey());
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
- if (data == null) {
- data = zookeeper.getData(dir + "/" + headNode.getKey(), null, null, true);
- }
- QueueEvent queueEvent = new QueueEvent(headNode.getKey(), data, null);
-
- updateLock.lockInterruptibly();
- try {
- knownChildren.put(headNode.getKey(), data);
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
-
+ QueueEvent queueEvent = new QueueEvent(dir + "/" + headNode, zookeeper.getData(dir + "/" + headNode,
+ null, null, true), null);
+ log.info("return {}", queueEvent.getId());
return queueEvent.getId();
} catch (KeeperException.NoNodeException e) {
- updateLock.lockInterruptibly();
- try {
- knownChildren.remove(headNode.getKey());
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
+ // Another client removed the node first, try next
+ log.info("no node found {}", dir + "/" + headNode);
}
}
return null;
@@ -402,12 +345,11 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
return true;
}
- private volatile WatchedEvent event = null;
- private volatile String id;
- private volatile byte[] bytes;
+ private WatchedEvent event = null;
+ private String id;
+ private byte[] bytes;
QueueEvent(String id, byte[] bytes, WatchedEvent event) {
- if (log.isDebugEnabled()) log.debug("Create QueueEvent with id {} {} {}", id, bytes != null ? bytes.length : 0, event);
this.id = id;
this.bytes = bytes;
this.event = event;
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java b/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
index 75cef2f..2ce6251 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
@@ -55,11 +55,11 @@ public class RecoveringCoreTermWatcher implements ZkShardTerms.CoreTermWatcher {
}
if (solrCore.getCoreDescriptor() == null || solrCore.getCoreDescriptor().getCloudDescriptor() == null) return true;
- String coreNodeName = solrCore.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
- if (terms.haveHighestTermValue(coreNodeName)) return true;
- if (terms.getTerm(coreNodeName) != null && lastTermDoRecovery.get() < terms.getTerm(coreNodeName)) {
- log.info("Start recovery on {} because core's term is less than leader's term", coreNodeName);
- lastTermDoRecovery.set(terms.getTerm(coreNodeName));
+ String coreName = solrCore.getCoreDescriptor().getName();
+ if (terms.haveHighestTermValue(coreName)) return true;
+ if (terms.getTerm(coreName) != null && lastTermDoRecovery.get() < terms.getTerm(coreName)) {
+ log.info("Start recovery on {} because core's term is less than leader's term", coreName);
+ lastTermDoRecovery.set(terms.getTerm(coreName));
solrCore.getUpdateHandler().getSolrCoreState().doRecovery(solrCore.getCoreContainer(), solrCore.getCoreDescriptor());
}
} catch (Exception e) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index f1ea34b..595344b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -69,6 +69,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
+import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@@ -83,6 +84,8 @@ import java.util.concurrent.atomic.AtomicInteger;
*/
public class RecoveryStrategy implements Runnable, Closeable {
+ private volatile CountDownLatch latch;
+
public static class Builder implements NamedListInitializedPlugin {
private NamedList args;
@@ -123,7 +126,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
private volatile RecoveryListener recoveryListener;
private final ZkController zkController;
private final String baseUrl;
- private volatile String coreZkNodeName;
private final ZkStateReader zkStateReader;
private volatile String coreName;
private final AtomicInteger retries = new AtomicInteger(0);
@@ -148,7 +150,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
zkController = cc.getZkController();
zkStateReader = zkController.getZkStateReader();
baseUrl = zkController.getBaseUrl();
- coreZkNodeName = cd.getCloudDescriptor().getCoreNodeName();
replicaType = cd.getCloudDescriptor().getReplicaType();
}
@@ -211,17 +212,25 @@ public class RecoveryStrategy implements Runnable, Closeable {
if (finalReplicationHandler != null) finalReplicationHandler.abortFetch();
});
+ closer.collect("latch", () -> {
+ try {
+ latch.countDown();
+ latch = null;
+ } catch (NullPointerException e) {
+ // expected
+ }
+ });
+
}
} finally {
core = null;
}
- log.warn("Stopping recovery for core=[{}] coreNodeName=[{}]", coreName, coreZkNodeName);
+ log.warn("Stopping recovery for core=[{}]", coreName);
//ObjectReleaseTracker.release(this);
}
final private void recoveryFailed(final SolrCore core,
- final ZkController zkController, final String baseUrl,
- final String shardZkNodeName, final CoreDescriptor cd) throws Exception {
+ final ZkController zkController, final String baseUrl, final CoreDescriptor cd) throws Exception {
SolrException.log(log, "Recovery failed - I give up.");
try {
if (zkController.getZkClient().isConnected()) {
@@ -239,16 +248,16 @@ public class RecoveryStrategy implements Runnable, Closeable {
*
* @lucene.experimental
*/
- protected String getReplicateLeaderUrl(ZkNodeProps leaderprops) {
- return new ZkCoreNodeProps(leaderprops).getCoreUrl();
+ protected String getReplicateLeaderUrl(Replica leaderprops) {
+ return leaderprops.getCoreUrl();
}
- final private void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops)
+ final private void replicate(String nodeName, SolrCore core, Replica leaderprops)
throws SolrServerException, IOException {
final String leaderUrl = getReplicateLeaderUrl(leaderprops);
- log.info("Attempting to replicate from [{}].", leaderUrl);
+ log.info("Attempting to replicate from [{}].", leaderprops);
// send commit
commitOnLeader(leaderUrl);
@@ -268,8 +277,10 @@ public class RecoveryStrategy implements Runnable, Closeable {
// always download the tlogs from the leader when running with cdcr enabled. We need to have all the tlogs
// to ensure leader failover doesn't cause missing docs on the target
- if (isClosed()) return; // we check closed on return
boolean success = false;
+
+ log.info("do replication fetch [{}].", solrParams);
+
IndexFetcher.IndexFetchResult result = replicationHandler.doFetch(solrParams, false);
if (result.getMessage().equals(IndexFetcher.IndexFetchResult.FAILED_BY_INTERRUPT_MESSAGE)) {
@@ -278,7 +289,10 @@ public class RecoveryStrategy implements Runnable, Closeable {
}
if (result.getSuccessful()) {
+ log.info("replication fetch reported as success");
success= true;
+ } else {
+ log.error("replication fetch reported as failed: {} {} {}", result.getMessage(), result, result.getException());
}
if (!success) {
@@ -319,6 +333,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
final private void commitOnLeader(String leaderUrl) throws SolrServerException,
IOException {
+ log.info("send commit to leader");
Http2SolrClient client = core.getCoreContainer().getUpdateShardHandler().getRecoveryOnlyClient();
UpdateRequest ureq = new UpdateRequest();
ureq.setBasePath(leaderUrl);
@@ -333,9 +348,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
@Override
final public void run() {
try {
- if (isClosed()) {
- return;
- }
+
// set request info for logging
@@ -389,7 +402,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
// though
try {
CloudDescriptor cloudDesc = this.coreDescriptor.getCloudDescriptor();
- ZkNodeProps leaderprops = zkStateReader.getLeaderRetry(
+ Replica leaderprops = zkStateReader.getLeaderRetry(
cloudDesc.getCollectionName(), cloudDesc.getShardId());
final String leaderBaseUrl = leaderprops.getStr(ZkStateReader.BASE_URL_PROP);
final String leaderCoreName = leaderprops.getStr(ZkStateReader.CORE_NAME_PROP);
@@ -488,7 +501,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
if (retries.incrementAndGet() >= maxRetries) {
SolrException.log(log, "Recovery failed - max retries exceeded (" + retries + ").");
try {
- recoveryFailed(core, zkController, baseUrl, coreZkNodeName, this.coreDescriptor);
+ recoveryFailed(core, zkController, baseUrl, this.coreDescriptor);
} catch (InterruptedException e) {
ParWork.propagateInterrupt(e);
return;
@@ -533,13 +546,14 @@ public class RecoveryStrategy implements Runnable, Closeable {
// TODO: perhaps make this grab a new core each time through the loop to handle core reloads?
public final void doSyncOrReplicateRecovery(SolrCore core) throws Exception {
+ log.info("Do peersync or replication recovery core={} collection={}", core.getName(), core.getCoreDescriptor().getCollectionName());
boolean successfulRecovery = false;
UpdateLog ulog;
ulog = core.getUpdateHandler().getUpdateLog();
if (ulog == null) {
SolrException.log(log, "No UpdateLog found - cannot recover.");
- recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
+ recoveryFailed(core, zkController, baseUrl,
this.coreDescriptor);
return;
}
@@ -659,12 +673,8 @@ public class RecoveryStrategy implements Runnable, Closeable {
break;
}
- sendPrepRecoveryCmd(leader.getBaseUrl(), leader.getCoreName(), slice);
+ sendPrepRecoveryCmd(leader.getBaseUrl(), leader.getName(), slice);
- if (isClosed()) {
- log.info("RecoveryStrategy has been closed");
- break;
- }
// we wait a bit so that any updates on the leader
// that started before they saw recovering state
@@ -704,7 +714,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
cloudDebugLog(core, "synced");
log.info("Replaying updates buffered during PeerSync.");
- replayFuture = replay(core);
+ replay(core);
// sync success
successfulRecovery = true;
@@ -714,10 +724,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
log.info("PeerSync Recovery was not successful - trying replication.");
}
- if (isClosed()) {
- log.info("RecoveryStrategy has been closed");
- break;
- }
log.info("Starting Replication Recovery.");
@@ -725,17 +731,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
replicate(zkController.getNodeName(), core, leader);
- if (isClosed()) {
- log.info("RecoveryStrategy has been closed");
- break;
- }
-
- replayFuture = replay(core);
-
- if (isClosed()) {
- log.info("RecoveryStrategy has been closed");
- break;
- }
+ replay(core);
log.info("Replication Recovery was successful.");
successfulRecovery = true;
@@ -780,17 +776,13 @@ public class RecoveryStrategy implements Runnable, Closeable {
// Or do a fall off retry...
try {
- if (isClosed()) {
- log.info("RecoveryStrategy has been closed");
- break;
- }
log.error("Recovery failed - trying again... ({})", retries);
if (retries.incrementAndGet() >= maxRetries) {
SolrException.log(log, "Recovery failed - max retries exceeded (" + retries + ").");
try {
- recoveryFailed(core, zkController, baseUrl, coreZkNodeName, this.coreDescriptor);
+ recoveryFailed(core, zkController, baseUrl, this.coreDescriptor);
} catch(InterruptedException e) {
ParWork.propagateInterrupt(e);
return;
@@ -842,7 +834,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
while (true) {
CloudDescriptor cloudDesc = coreDesc.getCloudDescriptor();
DocCollection docCollection = zkStateReader.getClusterState().getCollection(cloudDesc.getCollectionName());
- if (!isClosed() && mayPutReplicaAsDown && numTried == 1 && docCollection.getReplica(coreDesc.getCloudDescriptor().getCoreNodeName()).getState() == Replica.State.ACTIVE) {
+ if (!isClosed() && mayPutReplicaAsDown && numTried == 1 && docCollection.getReplica(coreDesc.getName()).getState() == Replica.State.ACTIVE) {
// this operation may take a long time, by putting replica into DOWN state, client won't query this replica
// zkController.publish(coreDesc, Replica.State.DOWN);
// TODO: We should be in recovery and ignored by queries?
@@ -871,7 +863,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
public static Runnable testing_beforeReplayBufferingUpdates;
- final private Future<RecoveryInfo> replay(SolrCore core)
+ final private void replay(SolrCore core)
throws InterruptedException, ExecutionException {
if (testing_beforeReplayBufferingUpdates != null) {
testing_beforeReplayBufferingUpdates.run();
@@ -882,7 +874,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
new ModifiableSolrParams());
core.getUpdateHandler().getUpdateLog().copyOverBufferingUpdates(new CommitUpdateCommand(req, false));
req.close();
- return null;
}
Future<RecoveryInfo> future = core.getUpdateHandler().getUpdateLog().applyBufferedUpdates();
if (future == null) {
@@ -911,8 +902,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
// solrcloud_debug
cloudDebugLog(core, "replayed");
-
- return future;
}
final private void cloudDebugLog(SolrCore core, String op) {
@@ -942,41 +931,71 @@ public class RecoveryStrategy implements Runnable, Closeable {
final private void sendPrepRecoveryCmd(String leaderBaseUrl, String leaderCoreName, Slice slice)
throws SolrServerException, IOException {
+ if (coreDescriptor.getCollectionName() == null) {
+ throw new IllegalStateException("Collection name cannot be null");
+ }
+
WaitForState prepCmd = new WaitForState();
- prepCmd.setCoreName(leaderCoreName);
+ prepCmd.setCoreName(coreName);
prepCmd.setNodeName(zkController.getNodeName());
- prepCmd.setCoreNodeName(coreZkNodeName);
prepCmd.setState(Replica.State.RECOVERING);
prepCmd.setCheckLive(true);
prepCmd.setOnlyIfLeader(true);
+ prepCmd.setCollection(coreDescriptor.getCollectionName());
+ prepCmd.setShardId(coreDescriptor.getCloudDescriptor().getShardId());
final Slice.State state = slice.getState();
if (state != Slice.State.CONSTRUCTION && state != Slice.State.RECOVERY && state != Slice.State.RECOVERY_FAILED) {
prepCmd.setOnlyIfLeaderActive(true);
}
+ log.info("Sending prep recovery command to {} for core {} params={}", leaderBaseUrl, leaderCoreName, prepCmd.getParams());
+
int conflictWaitMs = zkController.getLeaderConflictResolveWait();
int readTimeout = conflictWaitMs + Integer.parseInt(System.getProperty("prepRecoveryReadTimeoutExtraWait", "30000"));
// nocommit
try (Http2SolrClient client = new Http2SolrClient.Builder(leaderBaseUrl).withHttpClient(core.getCoreContainer().getUpdateShardHandler().
- getRecoveryOnlyClient()).idleTimeout(readTimeout).connectionTimeout(3000).markInternalRequest().build()) {
+ getRecoveryOnlyClient()).idleTimeout(readTimeout).markInternalRequest().build()) {
prepCmd.setBasePath(leaderBaseUrl);
log.info("Sending prep recovery command to [{}]; [{}]", leaderBaseUrl, prepCmd);
- Cancellable result = client.asyncRequest(prepCmd, null, new NamedListAsyncListener());
+ latch = new CountDownLatch(1);
+ Cancellable result = client.asyncRequest(prepCmd, null, new NamedListAsyncListener(latch));
prevSendPreRecoveryHttpUriRequest = result;
+ try {
+ latch.await();
+ } catch (InterruptedException e) {
+ ParWork.propagateInterrupt(e);
+ } finally {
+ prevSendPreRecoveryHttpUriRequest = null;
+ latch = null;
+ }
- client.waitForOutstandingRequests();
}
}
private static class NamedListAsyncListener implements AsyncListener<NamedList<Object>> {
+
+ private final CountDownLatch latch;
+
+ public NamedListAsyncListener(CountDownLatch latch) {
+ this.latch = latch;
+ }
+
@Override
public void onSuccess(NamedList<Object> entries) {
+ try {
+ latch.countDown();
+ } catch (NullPointerException e) {
+ }
}
@Override
public void onFailure(Throwable throwable) {
+ try {
+ latch.countDown();
+ } catch (NullPointerException e) {
+ }
}
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
index ae7b6c0..1be3027 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
@@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
+import net.sf.saxon.trans.Err;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.AlreadyClosedException;
@@ -68,7 +69,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
public ShardLeaderElectionContext(LeaderElector leaderElector,
final String shardId, final String collection,
- final String coreNodeName, ZkNodeProps props, ZkController zkController, CoreContainer cc) {
+ final String coreNodeName, Replica props, ZkController zkController, CoreContainer cc) {
super(coreNodeName, ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection
+ "/leader_elect/" + shardId, ZkStateReader.getShardLeadersPath(
collection, shardId), props,
@@ -122,16 +123,16 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
@Override
void runLeaderProcess(ElectionContext context, boolean weAreReplacement, int pauseBeforeStart) throws KeeperException,
InterruptedException, IOException {
- if (isClosed()) {
- return;
- }
String coreName = leaderProps.getStr(ZkStateReader.CORE_NAME_PROP);
+
+ log.info("Run leader process for shard election {}", coreName);
+
ActionThrottle lt;
try (SolrCore core = cc.getCore(coreName)) {
if (core == null) {
- // shutdown or removed
- return;
+ log.error("No SolrCore found, cannot become leader {}", coreName);
+ throw new SolrException(ErrorCode.SERVER_ERROR, "No SolrCore found, cannot become leader " + coreName);
}
MDCLoggingContext.setCore(core);
lt = core.getUpdateHandler().getSolrCoreState().getLeaderThrottle();
@@ -143,13 +144,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
if (log.isDebugEnabled()) log.debug("Running the leader process for shard={} and weAreReplacement={} and leaderVoteWait={}", shardId, weAreReplacement, leaderVoteWait);
- if (isClosed()) {
- // Solr is shutting down or the ZooKeeper session expired while waiting for replicas. If the later,
- // we cannot be sure we are still the leader, so we should bail out. The OnReconnect handler will
- // re-register the cores and handle a new leadership election.
- return;
- }
-
Replica.Type replicaType;
String coreNodeName;
boolean setTermToMax = false;
@@ -157,11 +151,10 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
CoreDescriptor cd = core.getCoreDescriptor();
CloudDescriptor cloudCd = cd.getCloudDescriptor();
replicaType = cloudCd.getReplicaType();
- coreNodeName = cloudCd.getCoreNodeName();
// should I be leader?
ZkShardTerms zkShardTerms = zkController.getShardTerms(collection, shardId);
- if (zkShardTerms.registered(coreNodeName) && !zkShardTerms.canBecomeLeader(coreNodeName)) {
- if (!waitForEligibleBecomeLeaderAfterTimeout(zkShardTerms, coreNodeName, leaderVoteWait)) {
+ if (zkShardTerms.registered(coreName) && !zkShardTerms.canBecomeLeader(coreName)) {
+ if (!waitForEligibleBecomeLeaderAfterTimeout(zkShardTerms, coreName, leaderVoteWait)) {
rejoinLeaderElection(core);
return;
} else {
@@ -170,22 +163,14 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
}
- if (isClosed()) {
- return;
- }
log.info("I may be the new leader - try and sync");
- if (isClosed()) {
- return;
- }
// nocommit
// we are going to attempt to be the leader
// first cancel any current recovery
core.getUpdateHandler().getSolrCoreState().cancelRecovery();
- if (isClosed()) {
- return;
- }
+
PeerSync.PeerSyncResult result = null;
boolean success = false;
try {
@@ -198,6 +183,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
if (!success) {
+
+ log.warn("Our sync attempt failed ulog={}", ulog);
boolean hasRecentUpdates = false;
if (ulog != null) {
// TODO: we could optimize this if necessary
@@ -206,19 +193,22 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
}
+ log.warn("Checking for recent versions in the update log", hasRecentUpdates);
if (!hasRecentUpdates) {
// we failed sync, but we have no versions - we can't sync in that case
// - we were active
// before, so become leader anyway if no one else has any versions either
if (result.getOtherHasVersions().orElse(false)) {
log.info("We failed sync, but we have no versions - we can't sync in that case. But others have some versions, so we should not become leader");
- success = false;
+ rejoinLeaderElection(core);
+ return;
} else {
log.info("We failed sync, but we have no versions - we can't sync in that case - we were active before, so become leader anyway");
success = true;
}
}
}
+ log.info("Our sync attempt succeeded");
// solrcloud_debug
if (log.isDebugEnabled()) {
@@ -236,92 +226,69 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
}
if (!success) {
- if (isClosed()) {
- log.info("Bailing on leader election, we are closed");
- return;
- }
log.info("Sync with potential leader failed, rejoining election ...");
rejoinLeaderElection(core);
return;
}
- if (!isClosed()) {
- try {
- if (replicaType == Replica.Type.TLOG) {
- // stop replicate from old leader
- zkController.stopReplicationFromLeader(coreName);
- if (weAreReplacement) {
-
- Future<UpdateLog.RecoveryInfo> future = core.getUpdateHandler().getUpdateLog().recoverFromCurrentLog();
- if (future != null) {
- log.info("Replaying tlog before become new leader");
- future.get();
- } else {
- log.info("New leader does not have old tlog to replay");
- }
-
+ try {
+ if (replicaType == Replica.Type.TLOG) {
+ // stop replicate from old leader
+ zkController.stopReplicationFromLeader(coreName);
+ if (weAreReplacement) {
+
+ Future<UpdateLog.RecoveryInfo> future = core.getUpdateHandler().getUpdateLog().recoverFromCurrentLog();
+ if (future != null) {
+ log.info("Replaying tlog before become new leader");
+ future.get();
+ } else {
+ log.info("New leader does not have old tlog to replay");
}
+
}
- // in case of leaderVoteWait timeout, a replica with lower term can win the election
- if (setTermToMax) {
- log.error("WARNING: Potential data loss -- Replica {} became leader after timeout (leaderVoteWait) " + "without being up-to-date with the previous leader", coreNodeName);
- zkController.getShardTerms(collection, shardId).setTermEqualsToLeader(coreNodeName);
- }
+ }
+ // in case of leaderVoteWait timeout, a replica with lower term can win the election
+ if (setTermToMax) {
+ log.error("WARNING: Potential data loss -- Replica {} became leader after timeout (leaderVoteWait) " + "without being up-to-date with the previous leader", coreName);
+ zkController.getShardTerms(collection, shardId).setTermEqualsToLeader(coreName);
+ }
- super.runLeaderProcess(context, weAreReplacement, 0);
+ super.runLeaderProcess(context, weAreReplacement, 0);
- assert shardId != null;
+ assert shardId != null;
- core.getCoreDescriptor().getCloudDescriptor().setLeader(true);
- publishActive(core);
- ZkNodeProps zkNodes = ZkNodeProps
- .fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(), ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.BASE_URL_PROP,
- leaderProps.get(ZkStateReader.BASE_URL_PROP), ZkStateReader.NODE_NAME_PROP, leaderProps.get(ZkStateReader.NODE_NAME_PROP), ZkStateReader.CORE_NAME_PROP,
- leaderProps.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.CORE_NODE_NAME_PROP, leaderProps.get(ZkStateReader.CORE_NODE_NAME_PROP), ZkStateReader.STATE_PROP,
- Replica.State.ACTIVE.toString());
- assert zkController != null;
- assert zkController.getOverseer() != null;
- zkController.getOverseer().offerStateUpdate(Utils.toJSON(zkNodes));
+ core.getCoreDescriptor().getCloudDescriptor().setLeader(true);
+ publishActive(core);
+ ZkNodeProps zkNodes = ZkNodeProps
+ .fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(), ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.BASE_URL_PROP,
+ leaderProps.get(ZkStateReader.BASE_URL_PROP), ZkStateReader.NODE_NAME_PROP, leaderProps.get(ZkStateReader.NODE_NAME_PROP), ZkStateReader.CORE_NAME_PROP,
+ leaderProps.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
+ assert zkController != null;
+ assert zkController.getOverseer() != null;
- log.info("I am the new leader: " + ZkCoreNodeProps.getCoreUrl(leaderProps) + " " + shardId);
+ log.info("Publish leader state");
+ zkController.getOverseer().offerStateUpdate(Utils.toJSON(zkNodes));
- } catch (AlreadyClosedException | InterruptedException e) {
- ParWork.propagateInterrupt("Already closed or interrupted, bailing..", e);
- return;
- } catch (Exception e) {
- SolrException.log(log, "There was a problem trying to register as the leader", e);
- ParWork.propagateInterrupt(e);
- if (isClosed()) {
- return;
- }
- if (e instanceof IOException || (e instanceof KeeperException && (!(e instanceof SessionExpiredException)))) {
+ log.info("I am the new leader: " + ZkCoreNodeProps.getCoreUrl(leaderProps) + " " + shardId);
- if (core == null) {
- if (log.isDebugEnabled()) log.debug("SolrCore not found:" + coreName + " in " + cc.getLoadedCoreNames());
- return;
- }
- core.getCoreDescriptor().getCloudDescriptor().setLeader(false);
+ } catch (AlreadyClosedException | InterruptedException e) {
+ ParWork.propagateInterrupt("Already closed or interrupted, bailing..", e);
+ throw new SolrException(ErrorCode.SERVER_ERROR, e);
+ } catch (SessionExpiredException e) {
+ throw e;
+ } catch (Exception e) {
+ SolrException.log(log, "There was a problem trying to register as the leader", e);
- // we could not publish ourselves as leader - try and rejoin election
- try {
- if (isClosed()) {
- return;
- }
- rejoinLeaderElection(core);
- } catch (Exception exc) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- }
+ core.getCoreDescriptor().getCloudDescriptor().setLeader(false);
- } else {
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- }
- }
- } else {
- log.info("Bailing on leader election, we are closed");
- cancelElection();
+ // we could not publish ourselves as leader - try and rejoin election
+
+ rejoinLeaderElection(core);
}
+
+ } catch (AlreadyClosedException e) {
+ log.info("CoreContainer is shutting down, won't become leader");
} finally {
MDCLoggingContext.clear();
}
@@ -334,7 +301,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
*/
private boolean waitForEligibleBecomeLeaderAfterTimeout(ZkShardTerms zkShardTerms, String coreNodeName, int timeout) throws InterruptedException {
long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS);
- while (!isClosed()) {
+ while (true) {
if (System.nanoTime() > timeoutAt) {
log.warn("After waiting for {}ms, no other potential leader was found, {} try to become leader anyway (core_term:{}, highest_term:{})",
timeout, coreNodeName, zkShardTerms.getTerm(coreNodeName), zkShardTerms.getHighestTerm());
@@ -349,7 +316,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
Thread.sleep(500L);
}
- return false;
}
/**
@@ -386,20 +352,11 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
private void rejoinLeaderElection(SolrCore core)
throws InterruptedException, KeeperException, IOException {
// remove our ephemeral and re join the election
- if (isClosed()) {
- log.debug("Not rejoining election because CoreContainer is closed");
- return;
- }
- log.info("There may be a better leader candidate than us - going back into recovery");
+ log.info("There may be a better leader candidate than us - will cancel election, rejoin election, and kick off recovery");
cancelElection();
- if (isClosed()) {
- log.debug("Not rejoining election because CoreContainer is closed");
- return;
- }
-
this.isClosed = false;
super.closed = false;
leaderElector.joinElection(this, true);
@@ -415,9 +372,5 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
return collection;
}
- @Override
- public boolean isClosed() {
- return isClosed || cc.isShutDown() || zkController.getZkClient().isConnected() == false;
- }
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
index 2e69314..f1e0269 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
@@ -28,6 +28,7 @@ import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.util.Utils;
@@ -49,7 +50,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
private volatile Integer leaderZkNodeParentVersion;
public ShardLeaderElectionContextBase(final String coreNodeName, String electionPath, String leaderPath,
- ZkNodeProps props, SolrZkClient zkClient) {
+ Replica props, SolrZkClient zkClient) {
super(coreNodeName, electionPath, leaderPath, props);
this.zkClient = zkClient;
}
@@ -90,7 +91,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
ops.add(Op.check(Paths.get(leaderPath).getParent().toString(), leaderZkNodeParentVersion));
ops.add(Op.delete(leaderSeqPath, -1));
ops.add(Op.delete(leaderPath, -1));
- zkClient.multi(ops);
+ zkClient.multi(ops, false);
} catch (KeeperException e) {
if (e instanceof NoNodeException) {
// okay
@@ -153,10 +154,11 @@ class ShardLeaderElectionContextBase extends ElectionContext {
List<String> errors = new ArrayList<>();
try {
- if (isClosed()) {
- log.info("Bailing on becoming leader, we are closed");
- return;
+
+ if (leaderSeqPath == null) {
+ throw new IllegalStateException("We have won as leader, but we have no leader election node known to us leaderPath " + leaderPath);
}
+
log.info("Creating leader registration node {} after winning as {} parent is {}", leaderPath, leaderSeqPath, parent);
List<Op> ops = new ArrayList<>(3);
@@ -170,7 +172,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
ops.add(Op.setData(parent, null, -1));
List<OpResult> results;
- results = zkClient.multi(ops);
+ results = zkClient.multi(ops, false);
log.info("Results from call {}", results);
Iterator<Op> it = ops.iterator();
for (OpResult result : results) {
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 d859512..4e88454 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
@@ -20,9 +20,11 @@ import java.io.Closeable;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
+import java.util.concurrent.TimeUnit;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.params.ModifiableSolrParams;
@@ -64,21 +66,16 @@ public class SyncStrategy implements Closeable {
public String baseUrl;
}
- public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, ZkNodeProps leaderProps) {
+ public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, Replica leaderProps) {
return sync(zkController, core, leaderProps, false);
}
- public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, ZkNodeProps leaderProps,
+ public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, Replica leaderProps,
boolean peerSyncOnlyWithActive) {
if (SKIP_AUTO_RECOVERY) {
return PeerSync.PeerSyncResult.success();
}
- if (isClosed()) {
- log.warn("Closed, skipping sync up.");
- return PeerSync.PeerSyncResult.failure();
- }
-
if (log.isInfoEnabled()) {
log.info("Sync replicas to {}", ZkCoreNodeProps.getCoreUrl(leaderProps));
}
@@ -91,16 +88,9 @@ public class SyncStrategy implements Closeable {
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()) {
- log.info("We have been closed, won't sync with replicas");
- return PeerSync.PeerSyncResult.failure();
- }
+ Replica leaderProps, boolean peerSyncOnlyWithActive) {
+
boolean success = false;
PeerSync.PeerSyncResult result = null;
assert core != null;
@@ -119,10 +109,6 @@ public class SyncStrategy implements Closeable {
SolrException.log(log, "Sync Failed", e);
}
try {
- if (isClosed()) {
- log.info("We have been closed, won't attempt to sync replicas back to leader");
- return PeerSync.PeerSyncResult.failure();
- }
if (success) {
log.info("Sync Success - now sync replicas to me");
@@ -144,13 +130,9 @@ public class SyncStrategy implements Closeable {
private PeerSync.PeerSyncResult syncWithReplicas(ZkController zkController, SolrCore core,
ZkNodeProps props, String collection, String shardId, boolean peerSyncOnlyWithActive) throws Exception {
- List<ZkCoreNodeProps> nodes = zkController.getZkStateReader()
- .getReplicaProps(collection, shardId,core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
-
- if (isClosed()) {
- log.info("We have been closed, won't sync with replicas");
- return PeerSync.PeerSyncResult.failure();
- }
+
+ List<Replica> nodes = zkController.getZkStateReader()
+ .getReplicaProps(collection, shardId,core.getCoreDescriptor().getName());
if (nodes == null) {
// I have no replicas
@@ -158,7 +140,7 @@ public class SyncStrategy implements Closeable {
}
List<String> syncWith = new ArrayList<>(nodes.size());
- for (ZkCoreNodeProps node : nodes) {
+ for (Replica node : nodes) {
syncWith.add(node.getCoreUrl());
}
@@ -174,20 +156,15 @@ public class SyncStrategy implements Closeable {
}
private void syncToMe(ZkController zkController, String collection,
- String shardId, ZkNodeProps leaderProps, CoreDescriptor cd,
+ String shardId, Replica leaderProps, CoreDescriptor cd,
int nUpdates) {
-
- if (isClosed()) {
- log.info("We have been closed, won't sync replicas to me.");
- return;
- }
-
+
// sync everyone else
// TODO: we should do this in parallel at least
- List<ZkCoreNodeProps> nodes = zkController
+ List<Replica> nodes = zkController
.getZkStateReader()
.getReplicaProps(collection, shardId,
- cd.getCloudDescriptor().getCoreNodeName());
+ cd.getName());
if (nodes == null) {
if (log.isInfoEnabled()) {
log.info("{} has no replicas", ZkCoreNodeProps.getCoreUrl(leaderProps));
@@ -195,14 +172,14 @@ public class SyncStrategy implements Closeable {
return;
}
- ZkCoreNodeProps zkLeader = new ZkCoreNodeProps(leaderProps);
- for (ZkCoreNodeProps node : nodes) {
+
+ for (Replica node : nodes) {
try {
if (log.isInfoEnabled()) {
log.info("{}: try and ask {} to sync", ZkCoreNodeProps.getCoreUrl(leaderProps), node.getCoreUrl());
}
- requestSync(node.getBaseUrl(), node.getCoreUrl(), zkLeader.getCoreUrl(), node.getCoreName(), nUpdates);
+ requestSync(node.getBaseUrl(), node.getCoreUrl(), leaderProps.getCoreUrl(), node.getName(), nUpdates);
} catch (Exception e) {
ParWork.propagateInterrupt(e);
@@ -210,23 +187,23 @@ public class SyncStrategy implements Closeable {
}
}
-
- for(;;) {
- ShardResponse srsp = shardHandler.takeCompletedOrError();
- if (srsp == null) break;
- boolean success = handleResponse(srsp);
- if (srsp.getException() != null) {
- SolrException.log(log, "Sync request error: " + srsp.getException());
- }
-
- if (!success) {
- if (log.isInfoEnabled()) {
- log.info("{}: Sync failed - replica ({}) should try to recover."
- , ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+ if (nodes.size() > 0) {
+ for (; ; ) {
+ ShardResponse srsp = shardHandler.takeCompletedOrError();
+ if (srsp == null) break;
+ boolean success = handleResponse(srsp);
+ if (srsp.getException() != null) {
+ SolrException.log(log, "Sync request error: " + srsp.getException());
}
- } else {
- if (log.isInfoEnabled()) {
- log.info("{}: sync completed with {}", ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+
+ if (!success) {
+ if (log.isInfoEnabled()) {
+ log.info("{}: Sync failed - replica ({}) should try to recover.", ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+ }
+ } else {
+ if (log.isInfoEnabled()) {
+ log.info("{}: sync completed with {}", ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+ }
}
}
}
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 a52567b..cdce9afa 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -111,7 +111,6 @@ import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTIONS_ZKNODE;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
@@ -206,7 +205,9 @@ public class ZkController implements Closeable, Runnable {
private static byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
- private final Map<ContextKey, ElectionContext> electionContexts = new ConcurrentHashMap<>(64, 0.75f, 16) {
+ private final Map<String, LeaderElector> leaderElectors = new ConcurrentHashMap<>(16);
+
+ private final Map<ContextKey, ElectionContext> electionContexts = new ConcurrentHashMap<>(16) {
@Override
public ElectionContext put(ContextKey key, ElectionContext value) {
if (ZkController.this.isClosed || cc.isShutDown()) {
@@ -216,7 +217,7 @@ public class ZkController implements Closeable, Runnable {
}
};
- private final Map<ContextKey, ElectionContext> overseerContexts = new ConcurrentHashMap<>(3, 0.75f, 1) {
+ private final Map<ContextKey, ElectionContext> overseerContexts = new ConcurrentHashMap<>() {
@Override
public ElectionContext put(ContextKey key, ElectionContext value) {
if (ZkController.this.isClosed || cc.isShutDown()) {
@@ -375,9 +376,9 @@ public class ZkController implements Closeable, Runnable {
public void closeLeaderContext(CoreDescriptor cd) {
String collection = cd.getCloudDescriptor().getCollectionName();
- final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
+ final String coreName = cd.getName();
- ContextKey contextKey = new ContextKey(collection, coreNodeName);
+ ContextKey contextKey = new ContextKey(collection, coreName);
ElectionContext context = electionContexts.get(contextKey);
if (context != null) {
try {
@@ -415,9 +416,14 @@ public class ZkController implements Closeable, Runnable {
@Override
public synchronized void command() {
- try (ParWork worker = new ParWork("disconnected", true)) {
+ try (ParWork worker = new ParWork("disconnected", true, true)) {
worker.collect("OverseerElectionContexts", overseerContexts.values());
- worker.collect("Overseer", ZkController.this.overseer);
+
+ worker.collect("Overseer", () -> {
+ if (ZkController.this.overseerElector != null) {
+ ZkController.this.overseerElector.getContext().close();
+ }
+ });
worker.collect("", () -> {
clearZkCollectionTerms();
});
@@ -431,13 +437,18 @@ public class ZkController implements Closeable, Runnable {
}
}
});
+ overseerContexts.clear();
+ electionContexts.clear();
zkClient.setAclProvider(zkACLProvider);
zkClient.getConnectionManager().setOnReconnect(new OnReconnect() {
@Override
public void command() throws SessionExpiredException {
synchronized (initLock) {
- if (cc.isShutDown() || !zkClient.isConnected()) return;
+ if (cc.isShutDown() || isClosed() || shudownCalled || !zkClient.isConnected()) {
+ log.info("skipping zk reconnect logic due to shutdown");
+ return;
+ }
log.info("ZooKeeper session re-connected ... refreshing core states after session expiration.");
try {
// recreate our watchers first so that they exist even on any problems below
@@ -457,17 +468,8 @@ public class ZkController implements Closeable, Runnable {
// start the overseer first as following code may need it's processing
- ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient ,overseer);
- ElectionContext prevContext = overseerContexts.put(new ContextKey("overseer", "overseer"), context);
- if (prevContext != null) {
- prevContext.close();
- }
- if (overseerElector != null) {
- ParWork.close(overseerElector.getContext());
- }
- overseerElector = new LeaderElector(zkClient, new ContextKey("overseer", "overseer"), overseerContexts);
- ZkController.this.overseer = new Overseer(cc.getUpdateShardHandler(),
- CommonParams.CORES_HANDLER_PATH, zkStateReader, ZkController.this, cloudConfig);
+ ElectionContext context = getOverseerContext();
+
overseerElector.setup(context);
overseerElector.joinElection(context, true);
@@ -477,7 +479,7 @@ public class ZkController implements Closeable, Runnable {
List<CoreDescriptor> descriptors = descriptorsSupplier.get();
// re register all descriptors
- try (ParWork parWork = new ParWork(this)) {
+ try (ParWork parWork = new ParWork(this, true, true)) {
if (descriptors != null) {
for (CoreDescriptor descriptor : descriptors) {
// TODO: we need to think carefully about what happens when it
@@ -487,7 +489,7 @@ public class ZkController implements Closeable, Runnable {
// with connection loss
try {
// unload solrcores that have been 'failed over'
- throwErrorIfReplicaReplaced(descriptor);
+ // throwErrorIfReplicaReplaced(descriptor);
parWork.collect(new RegisterCoreAsync(descriptor, true, true));
@@ -500,22 +502,21 @@ public class ZkController implements Closeable, Runnable {
// notify any other objects that need to know when the session was re-connected
- try (ParWork parWork = new ParWork(this)) {
+ try (ParWork parWork = new ParWork(this, true, true)) {
// the OnReconnect operation can be expensive per listener, so do that async in the background
reconnectListeners.forEach(listener -> {
try {
parWork.collect(new OnReconnectNotifyAsync(listener));
} catch (Exception exc) {
- SolrZkClient.checkInterrupted(exc);
// not much we can do here other than warn in the log
log.warn("Error when notifying OnReconnect listener {} after session re-connected.", listener, exc);
}
});
}
} catch (InterruptedException e) {
- log.warn("interrupted");
+ log.warn("interrupted", e);
} catch (SessionExpiredException e) {
- throw e;
+ log.warn("SessionExpiredException", e);
} catch (AlreadyClosedException e) {
log.info("Already closed");
return;
@@ -534,7 +535,7 @@ public class ZkController implements Closeable, Runnable {
}});
zkClient.setDisconnectListener(() -> {
if (isClosed()) return;
- try (ParWork worker = new ParWork("disconnected", true)) {
+ try (ParWork worker = new ParWork("disconnected", true, true)) {
if (zkClient.isConnected()) {
worker.collect(ZkController.this.overseerContexts);
} else {
@@ -554,6 +555,18 @@ public class ZkController implements Closeable, Runnable {
init();
}
+ private ElectionContext getOverseerContext() {
+ ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient, overseer);
+ ElectionContext prevContext = overseerContexts.put(new ContextKey("overseer", "overseer"), context);
+ if (prevContext != null) {
+ prevContext.close();
+ }
+ if (overseerElector != null) {
+ ParWork.close(overseerElector.getContext());
+ }
+ return context;
+ }
+
/**
* <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
* a helpful message regarding collection migration.</p>
@@ -614,7 +627,7 @@ public class ZkController implements Closeable, Runnable {
public void disconnect() {
log.info("disconnect");
this.dcCalled = true;
- try (ParWork closer = new ParWork(this, true)) {
+ try (ParWork closer = new ParWork(this, true, true)) {
closer.collect( "replicateFromLeaders", replicateFromLeaders);
if (getZkClient().getConnectionManager().isConnected()) {
@@ -647,36 +660,41 @@ public class ZkController implements Closeable, Runnable {
*/
public void close() {
log.info("Closing ZkController");
- assert closeTracker.close();
+ //assert closeTracker.close();
+
+ IOUtils.closeQuietly(overseerElector);
+
+ leaderElectors.forEach((s, leaderElector) -> IOUtils.closeQuietly(leaderElector));
+
+ if (overseer != null) {
+ overseer.closeAndDone();
+ }
+
this.shudownCalled = true;
this.isClosed = true;
try (ParWork closer = new ParWork(this, true, true)) {
+ closer.collect(overseer);
closer.collect(replicateFromLeaders);
closer.collect(electionContexts);
closer.collect(collectionToTerms);
closer.collect(sysPropsCacher);
closer.collect(cloudManager);
closer.collect(cloudSolrClient);
+ closer.collect(overseerContexts);
}
- try {
- if (overseer != null) {
- overseer.closeAndDone();
- }
- ParWork.close(overseerContexts);
- } finally {
- IOUtils.closeQuietly(zkStateReader);
+ IOUtils.closeQuietly(zkStateReader);
- if (closeZkClient) {
- IOUtils.closeQuietly(zkClient);
- }
+ if (closeZkClient) {
+ IOUtils.closeQuietly(zkClient);
+ }
- SolrLifcycleListener.removeShutdown(this);
+ SolrLifcycleListener.removeShutdown(this);
+
+ assert ObjectReleaseTracker.release(this);
- assert ObjectReleaseTracker.release(this);
- }
}
/**
@@ -694,7 +712,7 @@ public class ZkController implements Closeable, Runnable {
if (shard == null) return;
// if this replica is not a leader, it will be put in recovery state by the leader
- if (shard.getReplica(cd.getCloudDescriptor().getCoreNodeName()) != shard.getLeader()) return;
+ if (shard.getReplica(cd.getName()) != shard.getLeader()) return;
int numActiveReplicas = shard.getReplicas(
rep -> rep.getState() == Replica.State.ACTIVE
@@ -704,7 +722,7 @@ public class ZkController implements Closeable, Runnable {
// at least the leader still be able to search, we should give up leadership if other replicas can take over
if (numActiveReplicas >= 2) {
- String key = cd.getCollectionName() + ":" + cd.getCloudDescriptor().getCoreNodeName();
+ String key = cd.getCollectionName() + ":" + cd.getName();
//TODO better handling the case when delete replica was failed
if (replicasMetTragicEvent.putIfAbsent(key, tragicException) == null) {
log.warn("Leader {} met tragic exception, give up its leadership", key, tragicException);
@@ -714,7 +732,7 @@ public class ZkController implements Closeable, Runnable {
props.put(Overseer.QUEUE_OPERATION, "deletereplica");
props.put(COLLECTION_PROP, cd.getCollectionName());
props.put(SHARD_ID_PROP, shard.getName());
- props.put(REPLICA_PROP, cd.getCloudDescriptor().getCoreNodeName());
+ props.put(REPLICA_PROP, cd.getName());
getOverseerCollectionQueue().offer(Utils.toJSON(new ZkNodeProps(props)));
props.clear();
@@ -763,6 +781,7 @@ public class ZkController implements Closeable, Runnable {
cloudSolrClient = new CloudHttp2SolrClient.Builder(zkStateReader)
.withHttpClient(cc.getUpdateShardHandler().getTheSharedHttpClient())
.build();
+ cloudSolrClient.connect();
cloudManager = new SolrClientCloudManager(
new ZkDistributedQueueFactory(zkClient),
cloudSolrClient,
@@ -859,6 +878,10 @@ public class ZkController implements Closeable, Runnable {
return isClosed || getCoreContainer().isShutDown();
}
+ boolean isShudownCalled() {
+ return shudownCalled;
+ }
+
/**
* Create the zknodes necessary for a cluster to operate
*
@@ -884,7 +907,6 @@ public class ZkController implements Closeable, Runnable {
paths.put(Overseer.OVERSEER_ELECT + LeaderElector.ELECTION_NODE, null);
paths.put(Overseer.OVERSEER_QUEUE, null);
- paths.put(Overseer.OVERSEER_QUEUE_WORK, null);
paths.put(Overseer.OVERSEER_COLLECTION_QUEUE_WORK, null);
paths.put(Overseer.OVERSEER_COLLECTION_MAP_RUNNING, null);
paths.put(Overseer.OVERSEER_COLLECTION_MAP_COMPLETED, null);
@@ -1137,18 +1159,15 @@ public class ZkController implements Closeable, Runnable {
this.overseerConfigSetQueue = overseer.getConfigSetQueue(zkClient);
this.sysPropsCacher = new NodesSysPropsCacher(getSolrCloudManager().getNodeStateProvider(),
getNodeName(), zkStateReader);
-
+ overseerElector = new LeaderElector(this, new ContextKey("overseer", "overseer"), overseerContexts);
try (ParWork worker = new ParWork(this, false, true)) {
// start the overseer first as following code may need it's processing
worker.collect("startOverseer", () -> {
- LeaderElector overseerElector = new LeaderElector(zkClient, new ContextKey("overseer", "overseer"), overseerContexts);
- ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient, overseer);
- ElectionContext prevContext = overseerContexts.put(new ContextKey("overseer", "overseer"), context);
- if (prevContext != null) {
- prevContext.close();
- }
+ ElectionContext context = getOverseerContext();
+ log.info("Overseer setting up context {}", context.leaderProps);
overseerElector.setup(context);
try {
+ log.info("Overseer joining election {}", context.leaderProps);
overseerElector.joinElection(context, false);
} catch (KeeperException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, e);
@@ -1160,19 +1179,19 @@ public class ZkController implements Closeable, Runnable {
}
});
- worker.collect("publishDownState", () -> {
- try {
- Stat stat = zkClient.exists(ZkStateReader.LIVE_NODES_ZKNODE, null);
- if (stat != null && stat.getNumChildren() > 0) {
- publishDownStates();
- }
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- } catch (KeeperException e) {
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- }
- });
+// worker.collect("publishDownState", () -> {
+// try {
+// Stat stat = zkClient.exists(ZkStateReader.LIVE_NODES_ZKNODE, null);
+// if (stat != null && stat.getNumChildren() > 0) {
+// publishDownStates();
+// }
+// } catch (InterruptedException e) {
+// ParWork.propagateInterrupt(e);
+// throw new SolrException(ErrorCode.SERVER_ERROR, e);
+// } catch (KeeperException e) {
+// throw new SolrException(ErrorCode.SERVER_ERROR, e);
+// }
+// });
}
// Do this last to signal we're up.
createEphemeralLiveNode();
@@ -1193,64 +1212,8 @@ public class ZkController implements Closeable, Runnable {
private synchronized void shutdown() {
if (this.shudownCalled) return;
this.shudownCalled = true;
- Thread updaterThead = overseer.getUpdaterThread();
- log.info("Cluster shutdown initiated");
- if (updaterThead != null && updaterThead.isAlive()) {
- log.info("We are the Overseer, wait for others to shutdown");
-
- CountDownLatch latch = new CountDownLatch(1);
- List<String> children = null;
- try {
- children = zkClient.getChildren("/solr" + ZkStateReader.LIVE_NODES_ZKNODE, new Watcher() {
- @Override
- public void process(WatchedEvent event) {
- if (Event.EventType.None.equals(event.getType())) {
- return;
- }
- if (event.getType() == Event.EventType.NodeChildrenChanged) {
- Thread updaterThead = overseer.getUpdaterThread();
- if (updaterThead == null || !updaterThead.isAlive()) {
- log.info("We were the Overseer, but it seems not anymore, shutting down");
- latch.countDown();
- return;
- }
-
- try {
- List<String> children = zkClient.getChildren("/solr" + ZkStateReader.LIVE_NODES_ZKNODE, this, false);
- if (children.size() == 1) {
- latch.countDown();
- }
- } catch (KeeperException e) {
- log.error("Exception on proper shutdown", e);
- return;
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- return;
- }
- }
- }
- }, false);
- } catch (KeeperException e) {
- log.error("Time out waiting to see solr live nodes go down " + children.size());
- return;
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- return;
- }
- if (children.size() > 1) {
- boolean success = false;
- try {
- success = latch.await(10, TimeUnit.SECONDS);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- return;
- }
- if (!success) {
- log.error("Time out waiting to see solr live nodes go down " + children.size());
- }
- }
- }
+ log.info("Cluster shutdown initiated");
URL url = null;
try {
@@ -1361,33 +1324,17 @@ public class ZkController implements Closeable, Runnable {
return zkClient.exists(path);
}
- public void registerUnloadWatcher(String collection, String shardId, String coreNodeName, String name) {
- zkStateReader.registerDocCollectionWatcher(collection,
- new UnloadCoreOnDeletedWatcher(coreNodeName, shardId, name));
- }
-
- /**
- * Register shard with ZooKeeper.
- *
- * @return the shardId for the SolrCore
- */
- public String register(String coreName, final CoreDescriptor desc, boolean skipRecovery) throws Exception {
- try (SolrCore core = cc.getCore(coreName)) {
- return register(core, desc, false, false, skipRecovery);
- }
- }
-
- public String register(SolrCore core, final CoreDescriptor desc, boolean skipRecovery) throws Exception {
- return register(core, desc, false, false, skipRecovery);
+ public void registerUnloadWatcher(String collection, String shardId, String name) {
+ // nocommit - this thing is currently bad
+// zkStateReader.registerDocCollectionWatcher(collection,
+// new UnloadCoreOnDeletedWatcher(shardId, name));
}
-
public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores,
boolean afterExpiration, boolean skipRecovery) throws Exception {
try (SolrCore core = cc.getCore(coreName)) {
return register(core, desc, recoverReloadedCores, afterExpiration, skipRecovery);
}
-
}
/**
@@ -1395,56 +1342,63 @@ public class ZkController implements Closeable, Runnable {
*
* @return the shardId for the SolrCore
*/
- public String register(SolrCore core, final CoreDescriptor desc, boolean recoverReloadedCores,
+ private String register(SolrCore core, final CoreDescriptor desc, boolean recoverReloadedCores,
boolean afterExpiration, boolean skipRecovery) throws Exception {
MDCLoggingContext.setCoreDescriptor(cc, desc);
try {
- if (isClosed()) {
- throw new AlreadyClosedException();
- }
// pre register has published our down state
String coreName = core.getName();
final String baseUrl = getBaseUrl();
final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
final String collection = cloudDesc.getCollectionName();
final String shardId = cloudDesc.getShardId();
- final String coreZkNodeName = cloudDesc.getCoreNodeName();
- assert coreZkNodeName != null : "we should have a coreNodeName by now";
- log.info("Register SolrCore, baseUrl={} collection={}, shard={} coreNodeName={}", baseUrl, collection, shardId, coreZkNodeName);
-
-
- // check replica's existence in clusterstate first
+ log.info("Register SolrCore, core={} baseUrl={} collection={}, shard={} skipRecovery={}", coreName, baseUrl, collection, shardId, skipRecovery);
+ AtomicReference<DocCollection> coll = new AtomicReference<>();
try {
- zkStateReader.waitForState(collection, 30000,
- TimeUnit.MILLISECONDS, (collectionState) -> getReplicaOrNull(collectionState, shardId, coreZkNodeName) != null);
+ zkStateReader.waitForState(collection, 3, TimeUnit.SECONDS, (l, c) -> { // nocommit timeout
+// if (isClosed()) {
+// throw new AlreadyClosedException();
+// }
+
+ coll.set(c);
+ if (c == null) {
+ return false;
+ }
+ if (c.getReplica(coreName) != null) {
+ return true;
+ }
+ return false;
+ });
} catch (TimeoutException e) {
- throw new SolrException(ErrorCode.SERVER_ERROR, "Error registering SolrCore, timeout waiting for replica present in clusterstate");
+ throw new TimeoutException("Timeout waiting to see core " + coreName + " \n" + coll.get());
}
- Replica replica = getReplicaOrNull(zkStateReader.getClusterState().getCollectionOrNull(collection), shardId, coreZkNodeName);
+
+ Replica replica = getReplicaOrNull(zkStateReader.getClusterState().getCollectionOrNull(collection), shardId, coreName);
if (replica == null) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Error registering SolrCore, replica is removed from clusterstate");
}
-
- if (replica.getType() != Type.PULL) {
- getCollectionTerms(collection).register(cloudDesc.getShardId(), coreZkNodeName);
- }
+ // nocommit - we need to know what type we are without getting the clusterstate
+ /// if (replica.getType() != Type.PULL) {
+ log.info("Register terms for replica {}", coreName);
+ getCollectionTerms(collection).register(cloudDesc.getShardId(), coreName);
+ // }
ZkShardTerms shardTerms = getShardTerms(collection, cloudDesc.getShardId());
- log.info("Register replica - core:{} address:{} collection:{} shard:{}",
- coreName, baseUrl, collection, shardId);
-
+ log.info("Register replica - core:{} address:{} collection:{} shard:{}", coreName, baseUrl, collection, shardId);
+ //
try {
// If we're a preferred leader, insert ourselves at the head of the queue
boolean joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
if (replica.getType() != Type.PULL) {
+ // nocommit review
joinElection(desc, afterExpiration, joinAtHead);
} else if (replica.getType() == Type.PULL) {
if (joinAtHead) {
- log.warn("Replica {} was designated as preferred leader but it's type is {}, It won't join election", coreZkNodeName, Type.PULL);
+ log.warn("Replica {} was designated as preferred leader but it's type is {}, It won't join election", coreName, Type.PULL);
}
- log.debug("Replica {} skipping election because it's type is {}", coreZkNodeName, Type.PULL);
+ log.debug("Replica {} skipping election because it's type is {}", coreName, Type.PULL);
startReplicationFromLeader(coreName, false);
}
} catch (InterruptedException e) {
@@ -1453,24 +1407,18 @@ public class ZkController implements Closeable, Runnable {
} catch (KeeperException | IOException e) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
+ joinElection(desc, afterExpiration, false);
-
- // don't wait if we have closed
- if (cc.isShutDown()) {
- throw new AlreadyClosedException();
- }
-
-// getZkStateReader().waitForState(collection, 30, TimeUnit.SECONDS, (n,c) -> c != null && c.getLeader(shardId) != null && c.getLeader(shardId).getState().equals(
-// Replica.State.ACTIVE));
-
- // there should be no stale leader state at this point, dont hit zk directly
log.info("Wait to see leader for {}, {}", collection, shardId);
- String leaderUrl = zkStateReader.getLeaderUrl(collection, shardId, 60000);
+ Replica leader = zkStateReader.getLeaderRetry(collection, shardId, 10000);
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
- log.info("We are {} and leader is {}", ourUrl, leaderUrl);
- boolean isLeader = leaderUrl.equals(ourUrl);
- assert !(isLeader && replica.getType() == Type.PULL) : "Pull replica became leader!";
+ boolean isLeader = leader.getName() .equals(coreName);
+
+ log.info("We are {} and leader is {} isLeader={}", ourUrl, leader.getCoreUrl(), isLeader);
+
+ log.info("Check if we should recover isLeader={}", isLeader);
+ //assert !(isLeader && replica.getType() == Type.PULL) : "Pull replica became leader!";
try {
@@ -1495,7 +1443,7 @@ public class ZkController implements Closeable, Runnable {
if (slice.getState() != Slice.State.CONSTRUCTION || !isLeader) {
Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler().getUpdateLog().recoverFromLog();
if (recoveryFuture != null) {
- log.info("Replaying tlog for {} during startup... NOTE: This can take a while.", ourUrl);
+ log.info("Replaying tlog for {} during startup... NOTE: This can take a while.", core);
recoveryFuture.get(); // NOTE: this could potentially block for
// minutes or more!
// TODO: public as recovering in the mean time?
@@ -1507,16 +1455,12 @@ public class ZkController implements Closeable, Runnable {
}
}
}
- boolean didRecovery
- = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreZkNodeName, shardId, core, cc, afterExpiration);
+ boolean didRecovery = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreName, shardId, core, cc, afterExpiration);
if (!didRecovery) {
if (isTlogReplicaAndNotLeader) {
startReplicationFromLeader(coreName, true);
}
- if (!isLeader) {
- publish(desc, Replica.State.ACTIVE, true, false);
- }
}
if (replica.getType() != Type.PULL) {
@@ -1529,29 +1473,24 @@ public class ZkController implements Closeable, Runnable {
getZkStateReader().registerCore(cloudDesc.getCollectionName());
// the watcher is added to a set so multiple calls of this method will left only one watcher
// nocommit
- registerUnloadWatcher(cloudDesc.getCollectionName(), cloudDesc.getShardId(), cloudDesc.getCoreNodeName(), desc.getName());
+ registerUnloadWatcher(cloudDesc.getCollectionName(), cloudDesc.getShardId(), desc.getName());
} catch (Exception e) {
SolrZkClient.checkInterrupted(e);
unregister(coreName, desc, false);
throw e;
}
-
+ log.info("SolrCore Registered, core{} baseUrl={} collection={}, shard={}", coreName, baseUrl, collection, shardId);
return shardId;
} finally {
MDCLoggingContext.clear();
}
}
- private Replica getReplicaOrNull(DocCollection docCollection, String shard, String coreNodeName) {
+ private Replica getReplicaOrNull(DocCollection docCollection, String shard, String coreName) {
if (docCollection == null) return null;
- Slice slice = docCollection.getSlice(shard);
- if (slice == null) return null;
-
- Replica replica = slice.getReplica(coreNodeName);
- if (replica == null) return null;
- if (!getNodeName().equals(replica.getNodeName())) return null;
+ Replica replica = docCollection.getReplica(coreName);
return replica;
}
@@ -1630,7 +1569,7 @@ public class ZkController implements Closeable, Runnable {
* Get leader props directly from zk nodes.
* @throws SessionExpiredException on zk session expiration.
*/
- public ZkCoreNodeProps getLeaderProps(final String collection,
+ public Replica getLeaderProps(final String collection,
final String slice, int timeoutms) throws InterruptedException, SessionExpiredException {
return getLeaderProps(collection, slice, timeoutms, true);
}
@@ -1641,7 +1580,7 @@ public class ZkController implements Closeable, Runnable {
* @return leader props
* @throws SessionExpiredException on zk session expiration.
*/
- public ZkCoreNodeProps getLeaderProps(final String collection,
+ public Replica getLeaderProps(final String collection,
final String slice, int timeoutms, boolean failImmediatelyOnExpiration) throws InterruptedException, SessionExpiredException {
TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
Exception exp = null;
@@ -1651,7 +1590,8 @@ public class ZkController implements Closeable, Runnable {
byte[] data = zkClient.getData(ZkStateReader.getShardLeadersPath(collection, slice), null, null);
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(ZkNodeProps.load(data));
- return leaderProps;
+ // nocommit - right key for leader name?
+ return new Replica(leaderProps.getNodeProps().getStr("name"), leaderProps.getNodeProps().getProperties(), collection, slice);
} catch (Exception e) {
SolrZkClient.checkInterrupted(e);
@@ -1663,17 +1603,13 @@ public class ZkController implements Closeable, Runnable {
}
- private void joinElection(CoreDescriptor cd, boolean afterExpiration, boolean joinAtHead)
+ private boolean joinElection(CoreDescriptor cd, boolean afterExpiration, boolean joinAtHead)
throws InterruptedException, KeeperException, IOException {
- if (this.isClosed || cc.isShutDown()) {
- log.warn("cannot join election, closed");
- return;
- }
+ log.info("joinElection {}", cd.getName());
// look for old context - if we find it, cancel it
String collection = cd.getCloudDescriptor().getCollectionName();
- final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
- ContextKey contextKey = new ContextKey(collection, coreNodeName);
+ ContextKey contextKey = new ContextKey(collection, cd.getName());
ElectionContext prevContext = electionContexts.get(contextKey);
@@ -1688,18 +1624,20 @@ public class ZkController implements Closeable, Runnable {
props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
- props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
- ZkNodeProps ourProps = new ZkNodeProps(props);
+ Replica replica = new Replica(cd.getName(), props, null, null);
+ LeaderElector leaderElector;
+ synchronized (leaderElectors) {
+ leaderElector = leaderElectors.get(replica.getName());
+ if (leaderElector == null) {
+ leaderElector = new LeaderElector(this, contextKey, electionContexts);
+ leaderElectors.put(replica.getName(), leaderElector);
+ }
+ }
- LeaderElector leaderElector = new LeaderElector(zkClient, contextKey, electionContexts);
ElectionContext context = new ShardLeaderElectionContext(leaderElector, shardId,
- collection, coreNodeName, ourProps, this, cc);
+ collection, cd.getName(), replica, this, cc);
- if (this.isClosed || cc.isShutDown()) {
- context.close();
- return;
- }
prevContext = electionContexts.put(contextKey, context);
if (prevContext != null) {
prevContext.close();
@@ -1707,7 +1645,7 @@ public class ZkController implements Closeable, Runnable {
leaderElector.setup(context);
log.info("Joining election ...");
- leaderElector.joinElection(context, false, joinAtHead);
+ return leaderElector.joinElection(context, false, joinAtHead);
}
@@ -1735,7 +1673,7 @@ public class ZkController implements Closeable, Runnable {
core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
return true;
}
-
+ log.info("get shard terms {} {} {}", core.getName(), collection, shardId);
ZkShardTerms zkShardTerms = getShardTerms(collection, shardId);
if (zkShardTerms.registered(coreZkNodeName) && !zkShardTerms.canBecomeLeader(coreZkNodeName)) {
if (log.isInfoEnabled()) {
@@ -1789,8 +1727,6 @@ public class ZkController implements Closeable, Runnable {
String shardId = cd.getCloudDescriptor().getShardId();
- String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
-
Map<String,Object> props = new HashMap<>();
props.put(Overseer.QUEUE_OPERATION, "state");
props.put(ZkStateReader.STATE_PROP, state.toString());
@@ -1801,13 +1737,10 @@ public class ZkController implements Closeable, Runnable {
props.put(ZkStateReader.SHARD_ID_PROP, cd.getCloudDescriptor().getShardId());
props.put(ZkStateReader.COLLECTION_PROP, collection);
props.put(ZkStateReader.REPLICA_TYPE, cd.getCloudDescriptor().getReplicaType().toString());
- props.put(ZkStateReader.FORCE_SET_STATE_PROP, "false");
+
if (numShards != null) {
props.put(ZkStateReader.NUM_SHARDS_PROP, numShards.toString());
}
- if (coreNodeName != null) {
- props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
- }
try (SolrCore core = cc.getCore(cd.getName())) {
if (core != null && state == Replica.State.ACTIVE) {
ensureRegisteredSearcher(core);
@@ -1833,10 +1766,10 @@ public class ZkController implements Closeable, Runnable {
if (state == Replica.State.RECOVERING && cd.getCloudDescriptor().getReplicaType() != Type.PULL) {
// state is used by client, state of replica can change from RECOVERING to DOWN without needed to finish recovery
// by calling this we will know that a replica actually finished recovery or not
- getShardTerms(collection, shardId).startRecovering(coreNodeName);
+ getShardTerms(collection, shardId).startRecovering(cd.getName());
}
if (state == Replica.State.ACTIVE && cd.getCloudDescriptor().getReplicaType() != Type.PULL) {
- getShardTerms(collection, shardId).doneRecovering(coreNodeName);
+ getShardTerms(collection, shardId).doneRecovering(cd.getName());
}
ZkNodeProps m = new ZkNodeProps(props);
@@ -1879,8 +1812,8 @@ public class ZkController implements Closeable, Runnable {
}
public void unregister(String coreName, CoreDescriptor cd, boolean removeCoreFromZk) throws Exception {
+ log.info("Unregister core from zookeeper {}", coreName);
if (!zkClient.isConnected()) return;
- final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
final String collection = cd.getCloudDescriptor().getCollectionName();
ZkCollectionTerms ct = collectionToTerms.get(collection);
@@ -1890,7 +1823,7 @@ public class ZkController implements Closeable, Runnable {
zkStateReader.unregisterCore(collection);
- replicasMetTragicEvent.remove(collection+":"+coreNodeName);
+ replicasMetTragicEvent.remove(collection + ":" + coreName);
if (Strings.isNullOrEmpty(collection)) {
log.error("No collection was specified.");
@@ -1898,252 +1831,21 @@ public class ZkController implements Closeable, Runnable {
return;
}
final DocCollection docCollection = zkStateReader.getClusterState().getCollectionOrNull(collection);
- Replica replica = (docCollection == null) ? null : docCollection.getReplica(coreNodeName);
+ Replica replica = (docCollection == null) ? null : docCollection.getReplica(coreName);
if (replica == null || replica.getType() != Type.PULL) {
- ElectionContext context = electionContexts.remove(new ContextKey(collection, coreNodeName));
+ ElectionContext context = electionContexts.remove(new ContextKey(collection, coreName));
if (context != null) {
context.close();
}
}
- CloudDescriptor cloudDescriptor = cd.getCloudDescriptor();
- if (removeCoreFromZk) {
- ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
- OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
- ZkStateReader.NODE_NAME_PROP, getNodeName(),
- ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
- ZkStateReader.BASE_URL_PROP, getBaseUrl(),
- ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
- overseerJobQueue.offer(Utils.toJSON(m));
-
-// zkStateReader.waitForState(cloudDescriptor.getCollectionName(), 10, TimeUnit.SECONDS, (l,c) -> {
-// if (c == null) return true;
-// Slice slice = c.getSlice(cloudDescriptor.getShardId());
-// if (slice == null) return true;
-// Replica r = slice.getReplica(cloudDescriptor.getCoreNodeName());
-// if (r == null) return true;
-// return false;
-// });
- }
- }
-
- public void createCollection(String collection) throws Exception {
- ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
- CollectionParams.CollectionAction.CREATE.toLower(), ZkStateReader.NODE_NAME_PROP, getNodeName(),
- ZkStateReader.COLLECTION_PROP, collection);
- overseerJobQueue.offer(Utils.toJSON(m));
}
public ZkStateReader getZkStateReader() {
return zkStateReader;
}
- private void doGetShardIdAndNodeNameProcess(CoreDescriptor cd) {
- final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
-
- if (coreNodeName != null) {
- waitForShardId(cd);
- } else {
- // if no explicit coreNodeName, we want to match by base url and core name
- waitForCoreNodeName(cd);
- waitForShardId(cd);
- }
- }
-
- private void waitForCoreNodeName(CoreDescriptor cd) {
- if (log.isDebugEnabled()) log.debug("look for our core node name");
-
- AtomicReference<String> errorMessage = new AtomicReference<>();
- try {
- zkStateReader.waitForState(cd.getCollectionName(), 10, TimeUnit.SECONDS, (n, c) -> { // TODO: central timeout control
- if (c == null)
- return false;
- final Map<String,Slice> slicesMap = c.getSlicesMap();
- if (slicesMap == null) {
- return false;
- }
- for (Slice slice : slicesMap.values()) {
- for (Replica replica : slice.getReplicas()) {
-
- String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
- String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
-
- String msgNodeName = getNodeName();
- String msgCore = cd.getName();
-
- if (msgNodeName.equals(nodeName) && core.equals(msgCore)) {
- cd.getCloudDescriptor()
- .setCoreNodeName(replica.getName());
- return true;
- }
- }
- }
- return false;
- });
- } catch (TimeoutException | InterruptedException e) {
- String error = errorMessage.get();
- if (error == null)
- error = "";
- throw new NotInClusterStateException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName() + " " + error);
- }
- }
-
- private void waitForShardId(CoreDescriptor cd) {
- if (log.isDebugEnabled()) {
- log.debug("waitForShardId(CoreDescriptor cd={}) - start", cd);
- }
-
- AtomicReference<String> returnId = new AtomicReference<>();
- try {
- try {
- zkStateReader.waitForState(cd.getCollectionName(), 5, TimeUnit.SECONDS, (n, c) -> { // nocommit
- if (c == null) return false;
- String shardId = c.getShardId(cd.getCloudDescriptor().getCoreNodeName());
- if (shardId != null) {
- returnId.set(shardId);
- return true;
- }
- return false;
- });
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName());
- }
- } catch (TimeoutException e1) {
- log.error("waitForShardId(CoreDescriptor=" + cd + ")", e1);
-
- throw new SolrException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName());
- }
-
- final String shardId = returnId.get();
- if (shardId != null) {
- cd.getCloudDescriptor().setShardId(shardId);
-
- if (log.isDebugEnabled()) {
- log.debug("waitForShardId(CoreDescriptor) - end coreNodeName=" + cd.getCloudDescriptor().getCoreNodeName() + " shardId=" + shardId);
- }
- return;
- }
-
- throw new SolrException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName());
- }
-
- public String getCoreNodeName(CoreDescriptor descriptor) {
- String coreNodeName = descriptor.getCloudDescriptor().getCoreNodeName();
- if (coreNodeName == null && !genericCoreNodeNames) {
- // it's the default
- return getNodeName() + "_" + descriptor.getName();
- }
-
- return coreNodeName;
- }
-
- public void preRegister(CoreDescriptor cd) {
- log.info("PreRegister SolrCore, collection={}, shard={} coreNodeName={}", cd.getCloudDescriptor().getCollectionName(), cd.getCloudDescriptor().getShardId());
-
- CloudDescriptor cloudDesc = cd.getCloudDescriptor();
-
- String coreNodeName = getCoreNodeName(cd);
-
- // before becoming available, make sure we are not live and active
- // this also gets us our assigned shard id if it was not specified
- try {
- checkStateInZk(cd);
-
- // make sure the node name is set on the descriptor
- if (cloudDesc.getCoreNodeName() == null) {
- cloudDesc.setCoreNodeName(coreNodeName);
- }
- log.info("PreRegister found coreNodename of {}", coreNodeName);
-
- // publishState == false on startup
- if (isPublishAsDownOnStartup(cloudDesc)) {
- publish(cd, Replica.State.DOWN, false, true);
- }
- String collectionName = cd.getCloudDescriptor().getCollectionName();
- DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
- if (log.isDebugEnabled()) {
- log.debug(collection == null ?
- "Collection {} not visible yet, but flagging it so a watch is registered when it becomes visible" :
- "Registering watch for collection {}",
- collectionName);
- }
- } catch (KeeperException e) {
- log.error("", e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
- } catch (NotInClusterStateException e) {
- // make the stack trace less verbose
- throw e;
- } catch (Exception e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
- }
-
- doGetShardIdAndNodeNameProcess(cd);
-
- }
-
- /**
- * On startup, the node already published all of its replicas as DOWN,
- * we can skip publish the replica as down
- * @return Should publish the replica as down on startup
- */
- private boolean isPublishAsDownOnStartup(CloudDescriptor cloudDesc) {
- Replica replica = zkStateReader.getClusterState().getCollection(cloudDesc.getCollectionName())
- .getSlice(cloudDesc.getShardId())
- .getReplica(cloudDesc.getCoreNodeName());
- return !replica.getNodeName().equals(getNodeName());
- }
-
- private void checkStateInZk(CoreDescriptor cd) throws InterruptedException, NotInClusterStateException {
- CloudDescriptor cloudDesc = cd.getCloudDescriptor();
- String nodeName = cloudDesc.getCoreNodeName();
- if (nodeName == null) {
- nodeName = cloudDesc.getCoreNodeName();
- // verify that the repair worked.
- if (nodeName == null) {
- throw new SolrException(ErrorCode.SERVER_ERROR, "No coreNodeName for " + cd);
- }
- }
- final String coreNodeName = nodeName;
-
- if (cloudDesc.getShardId() == null) {
- throw new SolrException(ErrorCode.SERVER_ERROR, "No shard id for " + cd);
- }
- StringBuilder sb = new StringBuilder(256);
- AtomicReference<String> errorMessage = new AtomicReference<>();
- AtomicReference<DocCollection> collectionState = new AtomicReference<>();
- try {
- zkStateReader.waitForState(cd.getCollectionName(), WAIT_FOR_STATE, TimeUnit.SECONDS, (l, c) -> {
- collectionState.set(c);
- if (c == null) return false;
- Slice slice = c.getSlice(cloudDesc.getShardId());
- if (slice == null) {
- errorMessage.set("Invalid shard: " + cloudDesc.getShardId());
- return false;
- }
- Replica replica = slice.getReplica(coreNodeName);
- if (replica == null) {
- sb.setLength(0);
- slice.getReplicas().stream().forEach(replica1 -> sb.append(replica1.getName() + " "));
- errorMessage.set("coreNodeName " + coreNodeName + " does not exist in shard " + cloudDesc.getShardId() + ", ignore the exception if the replica was deleted. Found: " + sb.toString());
- return false;
- }
- return true;
- });
- } catch (TimeoutException e) {
- String error = errorMessage.get();
- if (error == null) error = "coreNodeName " + coreNodeName + " does not exist in shard " + cloudDesc.getShardId() + ", ignore the exception if the replica was deleted";
-
- throw new NotInClusterStateException(ErrorCode.SERVER_ERROR, error + "\n" + collectionState.get());
- }
-
- }
-
public static void linkConfSet(SolrZkClient zkClient, String collection, String confSetName) throws KeeperException, InterruptedException {
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
log.debug("Load collection config from:{}", path);
@@ -2299,6 +2001,14 @@ public class ZkController implements Closeable, Runnable {
}
}
+ public static String generateNodeName(final String url) {
+ try {
+ return URLEncoder.encode(trimLeadingAndTrailingSlashes(url), "UTF-8");
+ } catch (UnsupportedEncodingException e) {
+ throw new Error("JVM Does not seem to support UTF-8", e);
+ }
+ }
+
/**
* Utility method for trimming and leading and/or trailing slashes from
* its input. May return the empty string. May return null if and only
@@ -2317,11 +2027,19 @@ public class ZkController implements Closeable, Runnable {
return out;
}
- public void rejoinOverseerElection(String electionNode, boolean joinAtHead) {
- if (overseerElector == null) {
+ public void rejoinOverseerElection(boolean joinAtHead) {
+ boolean closeAndDone = false;
+ try {
+ closeAndDone = overseer.isCloseAndDone();
+ } catch (NullPointerException e) {
+ // okay
+ }
+
+ if (overseerElector == null || isClosed() || shudownCalled || closeAndDone) {
return;
}
try {
+ String electionNode = overseerElector.getContext().electionPath;
if (electionNode != null) {
// Check whether we came to this node by mistake
if ( overseerElector.getContext() != null && overseerElector.getContext().leaderSeqPath == null
@@ -2360,7 +2078,6 @@ public class ZkController implements Closeable, Runnable {
String collectionName = params.get(COLLECTION_PROP);
String shardId = params.get(SHARD_ID_PROP);
- String coreNodeName = params.get(CORE_NODE_NAME_PROP);
String coreName = params.get(CORE_NAME_PROP);
String electionNode = params.get(ELECTION_NODE_PROP);
String baseUrl = params.get(BASE_URL_PROP);
@@ -2370,16 +2087,21 @@ public class ZkController implements Closeable, Runnable {
log.info("Rejoin the shard leader election.");
- ContextKey contextKey = new ContextKey(collectionName, coreNodeName);
+ ContextKey contextKey = new ContextKey(collectionName, coreName);
ElectionContext prevContext = electionContexts.get(contextKey);
if (prevContext != null) prevContext.close();
- ZkNodeProps zkProps = new ZkNodeProps(BASE_URL_PROP, baseUrl, CORE_NAME_PROP, coreName, NODE_NAME_PROP, getNodeName(), CORE_NODE_NAME_PROP, coreNodeName);
+ Map<String, Object> props = new HashMap<>();
+ props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
+ props.put(ZkStateReader.CORE_NAME_PROP, coreName);
+ props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+
+ Replica replica = new Replica(coreName, props, null, null);
LeaderElector elect = ((ShardLeaderElectionContext) prevContext).getLeaderElector();
ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName,
- coreNodeName, zkProps, this, getCoreContainer());
+ coreName, replica, this, getCoreContainer());
context.leaderSeqPath = context.electionPath + LeaderElector.ELECTION_NODE + "/" + electionNode;
elect.setup(context);
@@ -2705,12 +2427,10 @@ public class ZkController implements Closeable, Runnable {
/** @lucene.internal */
public class UnloadCoreOnDeletedWatcher implements DocCollectionWatcher {
- String coreNodeName;
String shard;
String coreName;
- public UnloadCoreOnDeletedWatcher(String coreNodeName, String shard, String coreName) {
- this.coreNodeName = coreNodeName;
+ public UnloadCoreOnDeletedWatcher(String shard, String coreName) {
this.shard = shard;
this.coreName = coreName;
}
@@ -2724,7 +2444,7 @@ public class ZkController implements Closeable, Runnable {
if (getCoreContainer().getCoreDescriptor(coreName) == null) return true;
- boolean replicaRemoved = getReplicaOrNull(collectionState, shard, coreNodeName) == null;
+ boolean replicaRemoved = getReplicaOrNull(collectionState, shard, coreName) == null;
if (replicaRemoved) {
try {
log.info("Replica {} removed from clusterstate, remove it.", coreName);
@@ -2747,7 +2467,7 @@ public class ZkController implements Closeable, Runnable {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
UnloadCoreOnDeletedWatcher that = (UnloadCoreOnDeletedWatcher) o;
- return Objects.equals(coreNodeName, that.coreNodeName) &&
+ return
Objects.equals(shard, that.shard) &&
Objects.equals(coreName, that.coreName);
}
@@ -2755,7 +2475,7 @@ public class ZkController implements Closeable, Runnable {
@Override
public int hashCode() {
- return Objects.hash(coreNodeName, shard, coreName);
+ return Objects.hash(shard, coreName);
}
}
@@ -2775,7 +2495,7 @@ public class ZkController implements Closeable, Runnable {
for (Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
- Replica r = slice.getReplica(dcore.getCloudDescriptor().getCoreNodeName());
+ Replica r = slice.getReplica(dcore.getName());
if (r != null) {
return true;
}
@@ -2802,7 +2522,7 @@ public class ZkController implements Closeable, Runnable {
ZkStateReader.NODE_NAME_PROP, nodeName);
try {
overseer.getStateUpdateQueue().offer(Utils.toJSON(m));
- } catch (AlreadyClosedException e) {
+ } catch (AlreadyClosedException | InterruptedException e) {
ParWork.propagateInterrupt("Not publishing node as DOWN because a resource required to do so is already closed.", null, true);
return;
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java b/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java
index a413df8..c53b2f4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java
@@ -16,7 +16,6 @@
*/
package org.apache.solr.cloud;
-import java.io.File;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
@@ -27,7 +26,6 @@ import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
-import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@@ -37,15 +35,18 @@ import java.util.function.Predicate;
import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
-import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.ParWork;
+import org.apache.solr.cloud.overseer.NodeMutator;
+import org.apache.solr.cloud.overseer.ReplicaMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
+import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
@@ -82,6 +83,7 @@ public class ZkDistributedQueue implements DistributedQueue {
* We only bother setting a child watcher when the queue has no children in ZK.
*/
private static final Object _IMPLEMENTATION_NOTES = null;
+ public static final byte[] DATA = new byte[0];
final String dir;
@@ -92,22 +94,26 @@ public class ZkDistributedQueue implements DistributedQueue {
/**
* A lock that guards all of the mutable state that follows.
*/
- protected final ReentrantLock updateLock = new ReentrantLock();
+ private final ReentrantLock updateLock = new ReentrantLock();
/**
* Contains the last set of children fetched from ZK. Elements are removed from the head of
* this in-memory set as they are consumed from the queue. Due to the distributed nature
* of the queue, elements may appear in this set whose underlying nodes have been consumed in ZK.
- * Therefore, methods like peek have to double-check actual node existence, and methods
- * like poll must resolve any races by attempting to delete the underlying node.
+ * Therefore, methods like {@link #peek()} have to double-check actual node existence, and methods
+ * like {@link #poll()} must resolve any races by attempting to delete the underlying node.
*/
- protected volatile TreeMap<String,byte[]> knownChildren;
+ private TreeSet<String> knownChildren = new TreeSet<>();
/**
* Used to wait on ZK changes to the child list; you must hold {@link #updateLock} before waiting on this condition.
*/
private final Condition changed = updateLock.newCondition();
+ private boolean isDirty = true;
+
+ private int watcherCount = 0;
+
private final int maxQueueSize;
/**
@@ -133,22 +139,6 @@ public class ZkDistributedQueue implements DistributedQueue {
this.zookeeper = zookeeper;
this.stats = stats;
this.maxQueueSize = maxQueueSize;
-
- try {
- try {
- updateLock.lockInterruptibly();
- fetchZkChildren(null, null);
- } catch (KeeperException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
- }
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
}
/**
@@ -158,10 +148,10 @@ public class ZkDistributedQueue implements DistributedQueue {
* @return data at the first element of the queue, or null.
*/
@Override
- public byte[] peek(Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
+ public byte[] peek() throws KeeperException, InterruptedException {
Timer.Context time = stats.time(dir + "_peek");
try {
- return firstElement(acceptFilter);
+ return firstElement();
} finally {
time.stop();
}
@@ -175,8 +165,8 @@ public class ZkDistributedQueue implements DistributedQueue {
* @return data at the first element of the queue, or null.
*/
@Override
- public byte[] peek(Predicate<String> acceptFilter, boolean block) throws KeeperException, InterruptedException {
- return block ? peek(acceptFilter, Long.MAX_VALUE) : peek(acceptFilter);
+ public byte[] peek(boolean block) throws KeeperException, InterruptedException {
+ return block ? peek(Long.MAX_VALUE) : peek();
}
/**
@@ -187,40 +177,29 @@ public class ZkDistributedQueue implements DistributedQueue {
* @return data at the first element of the queue, or null.
*/
@Override
- public byte[] peek(Predicate<String> acceptFilter, long wait) throws KeeperException, InterruptedException {
- byte[] result = null;
-
+ public byte[] peek(long wait) throws KeeperException, InterruptedException {
+ Preconditions.checkArgument(wait > 0);
Timer.Context time;
if (wait == Long.MAX_VALUE) {
time = stats.time(dir + "_peek_wait_forever");
} else {
time = stats.time(dir + "_peek_wait" + wait);
}
-
- long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
-
- result = firstElement(acceptFilter);
- if (result != null) {
- return result;
- }
-
- ChildWatcher watcher = new ChildWatcher(acceptFilter);
- TreeMap<String,byte[]> foundChildren = fetchZkChildren(watcher, null);
-
- if (foundChildren.size() > 0) {
- result = firstElement(acceptFilter);
- return result;
- }
-
- TimeOut timeout = new TimeOut(waitNanos, TimeUnit.NANOSECONDS, TimeSource.NANO_TIME);
-
- waitForChildren(null, foundChildren, timeout, watcher);
- if (foundChildren.size() == 0) {
+ updateLock.lockInterruptibly();
+ try {
+ long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
+ while (waitNanos > 0) {
+ byte[] result = firstElement();
+ if (result != null) {
+ return result;
+ }
+ waitNanos = changed.awaitNanos(waitNanos);
+ }
return null;
+ } finally {
+ updateLock.unlock();
+ time.stop();
}
- result = firstElement(acceptFilter);
- return result;
-
}
/**
@@ -230,10 +209,10 @@ public class ZkDistributedQueue implements DistributedQueue {
* @return Head of the queue or null.
*/
@Override
- public byte[] poll(Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
+ public byte[] poll() throws KeeperException, InterruptedException {
Timer.Context time = stats.time(dir + "_poll");
try {
- return removeFirst(acceptFilter);
+ return removeFirst();
} finally {
time.stop();
}
@@ -245,10 +224,10 @@ public class ZkDistributedQueue implements DistributedQueue {
* @return The former head of the queue
*/
@Override
- public byte[] remove(Predicate<String> acceptFilter) throws NoSuchElementException, KeeperException{
+ public byte[] remove() throws NoSuchElementException, KeeperException, InterruptedException {
Timer.Context time = stats.time(dir + "_remove");
try {
- byte[] result = removeFirst(acceptFilter);
+ byte[] result = removeFirst();
if (result == null) {
throw new NoSuchElementException();
}
@@ -258,32 +237,48 @@ public class ZkDistributedQueue implements DistributedQueue {
}
}
- // TODO: use async
+// public void remove(Collection<String> paths) throws KeeperException, InterruptedException {
+//
+// if (log.isDebugEnabled()) log.debug("Remove paths from queue dir={} paths={}", dir, paths);
+//
+// if (paths.isEmpty()) {
+// if (log.isDebugEnabled()) log.debug("paths is empty, return");
+// return;
+// }
+//
+// List<String> fullPaths = new ArrayList<>(paths.size());
+// for (String node : paths) {
+// fullPaths.add(dir + "/" + node);
+// }
+//
+//
+// if (log.isDebugEnabled()) log.debug("delete nodes {}", fullPaths);
+// zookeeper.delete(fullPaths, false);
+// if (log.isDebugEnabled()) log.debug("after delete nodes");
+//
+// int cacheSizeBefore = knownChildren.size();
+// knownChildren.removeAll(paths);
+// if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
+// stats.setQueueLength(knownChildren.size());
+// } else {
+// // There are elements get deleted but not present in the cache,
+// // the cache seems not valid anymore
+// knownChildren.clear();
+// isDirty = true;
+// }
+// }
+
public void remove(Collection<String> paths) throws KeeperException, InterruptedException {
- if (log.isDebugEnabled()) log.debug("Remove paths from queue {} {}", dir, paths);
if (paths.isEmpty()) return;
-
- updateLock.lockInterruptibly();
- try {
- for (String path : paths) {
- knownChildren.remove(path);
- }
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
-
List<Op> ops = new ArrayList<>();
for (String path : paths) {
ops.add(Op.delete(dir + "/" + path, -1));
}
for (int from = 0; from < ops.size(); from += 1000) {
int to = Math.min(from + 1000, ops.size());
- List<Op> opList = ops.subList(from, to);
- if (opList.size() > 0) {
+ if (from < to) {
try {
- zookeeper.multi(ops.subList(from, to));
+ zookeeper.multi(ops.subList(from, to), true);
} catch (KeeperException.NoNodeException e) {
// don't know which nodes are not exist, so try to delete one by one node
for (int j = from; j < to; j++) {
@@ -298,6 +293,17 @@ public class ZkDistributedQueue implements DistributedQueue {
}
}
}
+
+ int cacheSizeBefore = knownChildren.size();
+ knownChildren.removeAll(paths);
+ if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
+ stats.setQueueLength(knownChildren.size());
+ } else {
+ // There are elements get deleted but not present in the cache,
+ // the cache seems not valid anymore
+ knownChildren.clear();
+ isDirty = true;
+ }
}
/**
@@ -306,39 +312,30 @@ public class ZkDistributedQueue implements DistributedQueue {
* @return The former head of the queue
*/
@Override
- public byte[] take(Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
+ public byte[] take() throws KeeperException, InterruptedException {
// Same as for element. Should refactor this.
Timer.Context timer = stats.time(dir + "_take");
updateLock.lockInterruptibly();
try {
- long waitNanos = TimeUnit.MILLISECONDS.toNanos(60000);
-
- byte[] result = removeFirst(acceptFilter);
- if (result != null) {
- return result;
- }
-
- ChildWatcher watcher = new ChildWatcher(acceptFilter);
- TreeMap<String,byte[]> foundChildren = fetchZkChildren(watcher, acceptFilter);
-
- TimeOut timeout = new TimeOut(waitNanos, TimeUnit.NANOSECONDS, TimeSource.NANO_TIME);
-
- waitForChildren( s -> s.startsWith(PREFIX) || acceptFilter.test(s), foundChildren, timeout, watcher);
- if (foundChildren.size() == 0) {
- return null;
- }
-
- result = removeFirst(acceptFilter);
- if (result != null) {
- return result;
+ while (true) {
+ byte[] result = removeFirst();
+ if (result != null) {
+ return result;
+ }
+ changed.await();
}
} finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
+ updateLock.unlock();
timer.stop();
}
- return null;
+ }
+
+ private static Set<String> OPERATIONS = new HashSet<>();
+ static {
+ OPERATIONS.add("state");
+ OPERATIONS.add("leader");
+ OPERATIONS.add("downnode");
+ OPERATIONS.add("updateshardstate");
}
/**
@@ -346,40 +343,50 @@ public class ZkDistributedQueue implements DistributedQueue {
* will be immediately visible when this method returns.
*/
@Override
- public void offer(byte[] data) throws KeeperException {
+ public void offer(byte[] data) throws KeeperException, InterruptedException {
+ // TODO change to accept json
+ Map json = (Map) Utils.fromJSON(data);
+
+ final String operation = (String) json.get(Overseer.QUEUE_OPERATION);
+// if (!OPERATIONS.contains(operation)) {
+// throw new IllegalArgumentException("unknown operation:" + operation + " contents:" + json);
+// }
Timer.Context time = stats.time(dir + "_offer");
- if (log.isDebugEnabled()) log.debug("Over item to queue {}", dir);
+
try {
- try {
- if (maxQueueSize > 0) {
- if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) {
- // If a max queue size is set, check it before creating a new queue item.
- Stat stat = zookeeper.exists(dir, null, true);
- if (stat == null) {
- // jump to the code below, which tries to create dir if it doesn't exist
- throw new KeeperException.NoNodeException();
- }
- int remainingCapacity = maxQueueSize - stat.getNumChildren();
- if (remainingCapacity <= 0) {
- throw new IllegalStateException("queue is full");
+ while (true) {
+ try {
+ if (maxQueueSize > 0) {
+ if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) { // nocommit review
+ // If a max queue size is set, check it before creating a new queue item.
+ Stat stat = zookeeper.exists(dir, null, true);
+ if (stat == null) {
+ // jump to the code below, which tries to create dir if it doesn't exist
+ throw new KeeperException.NoNodeException();
+ }
+ int remainingCapacity = maxQueueSize - stat.getNumChildren();
+ if (remainingCapacity <= 0) {
+ throw new IllegalStateException("queue is full");
+ }
+
+ // Allow this client to push up to 1% of the remaining queue capacity without rechecking.
+ offerPermits.set(remainingCapacity / 100);
}
+ }
- // Allow this client to push up to 1% of the remaining queue capacity without rechecking.
- offerPermits.set(remainingCapacity / 100);
+ // Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
+ // This will get set again when the watcher actually fires, but that's ok.
+ zookeeper.create(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL, true);
+ isDirty = true;
+ return;
+ } catch (KeeperException.NoNodeException e) {
+ try {
+ zookeeper.create(dir, DATA, CreateMode.PERSISTENT, true);
+ } catch (KeeperException.NodeExistsException ne) {
+ // someone created it
}
}
-
- // Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
- // This will get set again when the watcher actually fires, but that's ok.
- zookeeper.create(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL, true);
- return;
- } catch (KeeperException.NoNodeException e) {
- // someone created it
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new AlreadyClosedException(e);
}
-
} finally {
time.stop();
}
@@ -416,132 +423,72 @@ public class ZkDistributedQueue implements DistributedQueue {
/**
* Returns the name if the first known child node, or {@code null} if the queue is empty.
- * @return
+ * This is the only place {@link #knownChildren} is ever updated!
+ * The caller must double check that the actual node still exists, since the in-memory
+ * list is inherently stale.
*/
- private Map.Entry<String,byte[]> firstChild(boolean remove, Predicate<String> acceptFilter) {
+ private String firstChild(boolean remove, boolean refetchIfDirty) throws KeeperException, InterruptedException {
+ updateLock.lockInterruptibly();
try {
- updateLock.lockInterruptibly();
- try {
- // We always return from cache first, the cache will be cleared if the node is not exist
- if (!knownChildren.isEmpty()) {
- for (Map.Entry<String,byte[]> entry : knownChildren.entrySet()) {
- if (acceptFilter != null && acceptFilter.test(entry.getKey())) {
- continue;
- }
- if (remove) {
- knownChildren.remove(entry.getKey());
- try {
- zookeeper.delete(dir + "/" + entry.getKey(), -1);
- } catch (KeeperException.NoNodeException e) {
- if (log.isDebugEnabled()) log.debug("No node found for {}", entry.getKey());
- }
- return entry;
- }
- }
- return null;
- }
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
+ if (log.isDebugEnabled()) log.debug("firstChild isDirty={}, refetchIfDirty={}, remove={}", isDirty, refetchIfDirty, remove);
+ // We always return from cache first, the cache will be cleared if the node is not exist
+ if (!knownChildren.isEmpty() && !(isDirty && refetchIfDirty)) {
+ if (log.isDebugEnabled()) log.debug("returning first child");
+ return remove ? knownChildren.pollFirst() : knownChildren.first();
}
- fetchZkChildren(null, acceptFilter);
+ if (!isDirty && knownChildren.isEmpty()) {
+ if (log.isDebugEnabled()) log.debug("isDirty={} knownChildren is empty, returning null", isDirty);
+ return null;
+ }
- return null;
- } catch (InterruptedException | KeeperException e) {
- ParWork.propagateInterrupt(e);
- throw new AlreadyClosedException(e);
+ // Dirty, try to fetch an updated list of children from ZK.
+ // Only set a new watcher if there isn't already a watcher.
+
+ if (log.isDebugEnabled()) log.debug("isDirty={} fetch known children", isDirty);
+ ChildWatcher newWatcher = (watcherCount == 0) ? new ChildWatcher() : null;
+ knownChildren = fetchZkChildren(newWatcher);
+ if (newWatcher != null) {
+ watcherCount++; // watcher was successfully set
+ }
+ isDirty = false;
+ if (knownChildren.isEmpty()) {
+ if (log.isDebugEnabled()) log.debug("known children is empty return null");
+ return null;
+ }
+ if (log.isDebugEnabled()) log.debug("signal all and get first, remove={}", remove);
+ changed.signalAll();
+ return remove ? knownChildren.pollFirst() : knownChildren.first();
+ } finally {
+ updateLock.unlock();
}
}
/**
* Return the current set of children from ZK; does not change internal state.
*/
- TreeMap<String,byte[]> fetchZkChildren(Watcher watcher, Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
- if (log.isDebugEnabled()) log.debug("fetchZkChildren");
- TreeMap<String,byte[]> orderedChildren = new TreeMap<>();
- updateLock.lockInterruptibly();
- try {
- if (knownChildren != null && !knownChildren.isEmpty()) {
- Set<Map.Entry<String,byte[]>> entrySet = knownChildren.entrySet();
- for (Map.Entry<String,byte[]> entry : entrySet) {
- String childName = entry.getKey();
+ TreeSet<String> fetchZkChildren(Watcher watcher) throws InterruptedException, KeeperException {
+ while (true) {
+ try {
+ TreeSet<String> orderedChildren = new TreeSet<>();
+ if (log.isDebugEnabled()) log.debug("fetchZkChildren {}", dir);
+ List<String> childNames = zookeeper.getChildren(dir, watcher, true);
+ if (log.isDebugEnabled()) log.debug("got {} nodes", childNames);
+ stats.setQueueLength(childNames.size());
+ for (String childName : childNames) {
// Check format
- if (!childName.startsWith(PREFIX)) {
-
- // responses can be written to same queue with different naming scheme
- if (log.isDebugEnabled()) log.debug("Filtering child out by prefix name=: {}, prefix={}", childName, PREFIX);
+ if (!childName.regionMatches(0, PREFIX, 0, PREFIX.length())) {
+ log.debug("Found child node with improper name: {}", childName);
continue;
}
- if (acceptFilter != null && acceptFilter.test(childName)) {
- if (log.isDebugEnabled()) log.debug("Found child that matched exclude filter: {}", dir + "/" + childName);
- continue;
- }
- if (log.isDebugEnabled()) log.debug("found: {}", childName);
- orderedChildren.put(childName, entry.getValue());
- }
- }
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
-
- if (!orderedChildren.isEmpty()) {
- if (log.isDebugEnabled()) log.debug("found children from fetch {}", orderedChildren.size());
- return orderedChildren;
- }
- if (log.isDebugEnabled()) log.debug("found no children to fetch");
- TreeMap<String,byte[]> remoteKnownChildren = new TreeMap<>();
- try {
- List<String> childNames = zookeeper.getChildren(dir, watcher, true);
- stats.setQueueLength(childNames.size());
- for (String childName : childNames) {
- if (log.isDebugEnabled()) log.debug("Examine child: {} out of children={} acceptFilter={}", childName, childNames.size(), acceptFilter);
- // Check format
- if (!childName.startsWith(PREFIX)) {
- // responses can be written to same queue with different naming scheme
- if (log.isDebugEnabled()) log.debug("Excluding child by prefix: {}", childName);
- continue;
- }
- remoteKnownChildren.put(childName, null);
- if (acceptFilter != null && acceptFilter.test(childName)) {
- if (log.isDebugEnabled()) log.debug("Found child that matched exclude filter: {}", childName);
- continue;
- }
- if (log.isDebugEnabled()) log.debug("Add child to fetched children: {}", childName);
- orderedChildren.put(childName, null);
- }
- if (log.isDebugEnabled()) log.debug("found {} remote children", remoteKnownChildren.size());
- updateKnownChildren(remoteKnownChildren);
- if (log.isDebugEnabled()) log.debug("returning {} matched children", orderedChildren.size());
- return orderedChildren;
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new AlreadyClosedException(e);
- }
- }
-
- private void updateKnownChildren(TreeMap<String,byte[]> children) {
- if (log.isDebugEnabled()) log.debug("Update known children size={}", children.size());
- TreeMap<String,byte[]> newKnownChildren = new TreeMap<>();
- updateLock.lock();
- try {
- Set<Map.Entry<String,byte[]>> entrySet = children.entrySet();
- for (Map.Entry<String,byte[]> entry : entrySet) {
- String childName = entry.getKey();
- byte[] data = entry.getValue();
- if (data == null) {
- data = knownChildren == null ? null : knownChildren.get(childName);
+ orderedChildren.add(childName);
}
- newKnownChildren.put(childName, data);
+ if (log.isDebugEnabled()) log.debug("returning {}", orderedChildren);
+ return orderedChildren;
+ } catch (KeeperException.NoNodeException e) {
+ zookeeper.makePath(dir, false, true);
+ // go back to the loop and try again
}
- knownChildren = newKnownChildren;
- } catch (Exception e) {
- log.error("", e);
- } finally {
- updateLock.unlock();
}
}
@@ -553,139 +500,67 @@ public class ZkDistributedQueue implements DistributedQueue {
*/
@Override
public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
- if (log.isDebugEnabled()) log.debug("peekElements {} {}", max, acceptFilter);
- List<Pair<String,byte[]>> result = null;
- ChildWatcher watcher = new ChildWatcher(acceptFilter);
- TreeMap<String,byte[]> foundChildren = fetchZkChildren(watcher, acceptFilter);
+ List<String> foundChildren = new ArrayList<>();
long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
- TimeOut timeout = new TimeOut(waitNanos, TimeUnit.NANOSECONDS, TimeSource.NANO_TIME);
- try {
- if (foundChildren.size() == 0) {
- if (log.isDebugEnabled()) log.debug("found no children, watch for them excludeFilter={}", acceptFilter);
- waitForChildren(acceptFilter, foundChildren, timeout, watcher);
- }
+ if (log.isDebugEnabled()) log.debug("peekElements {} {}", max, waitMillis);
+ boolean first = true;
+ while (true) {
+ // Trigger a refresh, but only force it if this is not the first iteration.
+ if (log.isDebugEnabled()) log.debug("call first child {}", !first);
+ firstChild(false, first);
- // Technically we could restart the method if we fasil to actually obtain any valid children
- // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
- // much more sophisticated waitNanos tracking.
- if (log.isDebugEnabled()) log.debug("found children to process {}", foundChildren.size());
- result = Collections.synchronizedList(new ArrayList<>(foundChildren.size()));
- Set<String> dataPaths = new HashSet<>();
- for (Map.Entry<String,byte[]> child : foundChildren.entrySet()) {
- if (result.size() >= max) {
+ updateLock.lockInterruptibly();
+ try {
+ for (String child : knownChildren) {
+ if (acceptFilter.test(child)) {
+ foundChildren.add(child);
+ }
+ }
+ if (!foundChildren.isEmpty()) {
break;
}
-
- byte[] data = child.getValue();
- if (data == null) {
- updateLock.lockInterruptibly();
- try {
- data = knownChildren.get(child.getKey());
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
+ if (waitNanos <= 0) {
+ break;
}
- if (data == null) {
- // nocommit - lets not reget what we have in knownChildren, also, use asyncp
- dataPaths.add(dir + "/" + child.getKey());
- if (log.isDebugEnabled()) log.debug("get data for child={}", child.getKey());
- } else {
- if (log.isDebugEnabled()) log.debug("found data locally already {}", child.getKey());
- result.add(new Pair<>(child.getKey(), data));
+ // If this is our first time through, force a refresh before waiting.
+ if (first) {
+ first = false;
+ continue;
}
- }
- if (log.isDebugEnabled()) log.debug("fetch data for paths {}", dataPaths);
- Map<String,byte[]> dataMap = zookeeper.getData(dataPaths);
- updateLock.lockInterruptibly();
- List<Pair<String,byte[]>> finalResult = result;
- try {
- dataMap.forEach((k, bytes) -> {
- finalResult.add(new Pair<>(k, bytes));
- if (bytes != null) {
- knownChildren.put(new File(k).getName(), bytes);
- }
- });
+ waitNanos = changed.awaitNanos(waitNanos);
} finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
+ updateLock.unlock();
}
- if (log.isDebugEnabled()) log.debug("peek elements returning {} nodes", finalResult.size());
- return new ArrayList<>(finalResult);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new AlreadyClosedException(e);
- }
- }
-
- private void waitForChildren(Predicate<String> acceptFilter, TreeMap<String,byte[]> foundChildren, TimeOut timeout, ChildWatcher watcher) throws InterruptedException, KeeperException {
- if (log.isDebugEnabled()) log.debug("wait for children ... {}ms", timeout.getInterval(TimeUnit.MILLISECONDS));
- updateLock.lockInterruptibly();
- try {
- for (Map.Entry<String,byte[]> child : knownChildren.entrySet()) {
- if (!child.getKey().startsWith(PREFIX) && (acceptFilter == null || !acceptFilter.test(child.getKey()))) {
- foundChildren.put(child.getKey(), child.getValue());
- }
- }
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
+ if (!foundChildren.isEmpty()) {
+ break;
}
}
- if (!foundChildren.isEmpty()) {
- if (log.isDebugEnabled()) log.debug("Found new children ... {}", foundChildren.size());
- return;
- }
- if (timeout.hasTimedOut()) {
- if (log.isDebugEnabled()) log.debug("0 wait time and no children found, return");
- return;
- }
- TreeMap<String,byte[]> fc = null;
- while (fc == null || fc.isEmpty()) {
- fc = fetchZkChildren(watcher, acceptFilter);
- if (!fc.isEmpty()) {
- foundChildren.putAll(fc);
- return;
+
+ // Technically we could restart the method if we fail to actually obtain any valid children
+ // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
+ // much more sophisticated waitNanos tracking.
+ List<Pair<String, byte[]>> result = new ArrayList<>();
+ for (String child : foundChildren) {
+ if (result.size() >= max) {
+ break;
}
- updateLock.lockInterruptibly();
try {
+ byte[] data = zookeeper.getData(dir + "/" + child, null, null, true);
+ result.add(new Pair<>(child, data));
+ } catch (KeeperException.NoNodeException e) {
+ // Another client deleted the node first, remove the in-memory and continue.
+ updateLock.lockInterruptibly();
try {
- changed.await(Math.min(timeout.getInterval(TimeUnit.MILLISECONDS), 5000), TimeUnit.MILLISECONDS);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- }
- if (zookeeper.isClosed() || !zookeeper.isConnected()) {
- throw new AlreadyClosedException();
- }
- if (timeout.hasTimedOut()) {
- return;
- }
- for (Map.Entry<String,byte[]> child : knownChildren.entrySet()) {
- if (acceptFilter == null || !acceptFilter.test(child.getKey())) {
- foundChildren.put(child.getKey(), child.getValue());
- }
- }
- if (!foundChildren.isEmpty()) {
- try {
- if (log.isDebugEnabled()) log.debug("Remove watches for {}");
- zookeeper.getSolrZooKeeper().removeAllWatches(dir, Watcher.WatcherType.Children, false);
- } catch (Exception e) {
- log.info(e.getMessage());
- }
- return;
- }
-
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
+ knownChildren.remove(child);
+ } finally {
updateLock.unlock();
}
}
}
+ return result;
}
/**
@@ -693,137 +568,96 @@ public class ZkDistributedQueue implements DistributedQueue {
*
* @return the data at the head of the queue.
*/
- private byte[] firstElement(Predicate<String> acceptFilter) throws KeeperException {
- try {
-
- Map.Entry<String,byte[]> firstChild = firstChild(false, acceptFilter);
+ private byte[] firstElement() throws KeeperException, InterruptedException {
+ while (true) {
+ if (log.isDebugEnabled()) log.debug("calling firstElement");
+ String firstChild = firstChild(false, false);
if (firstChild == null) {
+ if (log.isDebugEnabled()) log.debug("return null");
return null;
}
- byte[] data;
- updateLock.lockInterruptibly();
- try {
- data = knownChildren.get(firstChild.getKey());
- if (data != null) {
- return data;
- }
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
-
try {
- data = zookeeper.getData(dir + "/" + firstChild.getKey(), null, null, true);
- if (data != null) {
- updateLock.lockInterruptibly();
- try {
- knownChildren.put(firstChild.getKey(), data);
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
- }
-
- return data;
+ if (log.isDebugEnabled()) log.debug("get data {}", dir + "/" + firstChild);
+ return zookeeper.getData(dir + "/" + firstChild, null, null, true);
} catch (KeeperException.NoNodeException e) {
+ if (log.isDebugEnabled()) log.debug("no node found {}, clear known children and set dirty", dir + "/" + firstChild);
+ // Another client deleted the node first, remove the in-memory and retry.
updateLock.lockInterruptibly();
try {
- knownChildren.remove(firstChild.getKey());
+ // Efficient only for single-consumer
+ knownChildren.clear();
+ isDirty = true;
} finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
+ updateLock.unlock();
}
- return null;
}
-
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new AlreadyClosedException(e);
}
}
- private byte[] removeFirst(Predicate<String> acceptFilter) throws KeeperException {
- try {
- Map.Entry<String,byte[]> firstChild = firstChild(true, acceptFilter);
+ private byte[] removeFirst() throws KeeperException, InterruptedException {
+ while (true) {
+ String firstChild = firstChild(true, false);
if (firstChild == null) {
return null;
}
-
- byte[] data;
- updateLock.lockInterruptibly();
try {
- data = knownChildren.remove(firstChild.getKey());
- } finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
- }
-
- try {
- String path = dir + "/" + firstChild.getKey();
- if (data == null) {
- data = zookeeper.getData(path, null, null, true);
- }
+ String path = dir + "/" + firstChild;
+ byte[] result = zookeeper.getData(path, null, null, true);
zookeeper.delete(path, -1, true);
+ stats.setQueueLength(knownChildren.size());
+ return result;
+ } catch (KeeperException.NoNodeException e) {
+ // Another client deleted the node first, remove the in-memory and retry.
updateLock.lockInterruptibly();
try {
- knownChildren.remove(firstChild.getKey());
-
+ // Efficient only for single-consumer
+ knownChildren.clear();
+ isDirty = true;
} finally {
- if (updateLock.isHeldByCurrentThread()) {
- updateLock.unlock();
- }
+ updateLock.unlock();
}
- return data;
- } catch (KeeperException.NoNodeException e) {
- return null;
}
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new AlreadyClosedException(e);
}
}
- @VisibleForTesting long watcherCount() throws InterruptedException {
- return 0;
+ @VisibleForTesting int watcherCount() throws InterruptedException {
+ updateLock.lockInterruptibly();
+ try {
+ return watcherCount;
+ } finally {
+ updateLock.unlock();
+ }
}
- @VisibleForTesting class ChildWatcher implements Watcher {
-
- private final Predicate<String> acceptFilter;
+ @VisibleForTesting boolean isDirty() throws InterruptedException {
+ updateLock.lockInterruptibly();
+ try {
- public ChildWatcher(Predicate<String> acceptFilter) {
- this.acceptFilter = acceptFilter;
+ if (log.isDebugEnabled()) log.debug("isDirty {}", isDirty);
+ return isDirty;
+ } finally {
+ updateLock.unlock();
}
+ }
+
+ @VisibleForTesting class ChildWatcher implements Watcher {
@Override
public void process(WatchedEvent event) {
+
+ if (log.isDebugEnabled()) log.debug("ChildWatcher fired {} {}", event.getType(), event.getPath());
// session events are not change events, and do not remove the watcher; except for Expired
- if (Event.EventType.None.equals(event.getType())) {
+ if (Event.EventType.None.equals(event.getType()) && !Event.KeeperState.Expired.equals(event.getState())) {
return;
}
- if (log.isDebugEnabled()) log.debug("DistributedQueue changed {} {}", event.getPath(), event.getType());
-
- // nocommit - all the nodes are watching this currently instead of just the Overseer
- if (event.getType() == Event.EventType.NodeChildrenChanged) {
- updateLock.lock();
- try {
- TreeMap<String,byte[]> found = fetchZkChildren(null, acceptFilter);
- if (!found.isEmpty()) {
- changed.signalAll();
- } else {
- fetchZkChildren(this, acceptFilter);
- }
- } catch (KeeperException | InterruptedException e) {
- log.error("", e);
- } finally {
- updateLock.unlock();
- }
- } else {
-
+ updateLock.lock();
+ try {
+ isDirty = true;
+ watcherCount--;
+ // optimistically signal any waiters that the queue may not be empty now, so they can wake up and retry
+ changed.signalAll();
+ } finally {
+ updateLock.unlock();
}
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
index 83be9ef..6dd410a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
@@ -186,7 +186,7 @@ public class ZkShardTerms implements AutoCloseable{
listeners.removeIf(coreTermWatcher -> !coreTermWatcher.onTermChanged(terms.get()));
numListeners = listeners.size();
- return removeTerm(cd.getCloudDescriptor().getCoreNodeName()) || numListeners == 0;
+ return removeTerm(cd.getName()) || numListeners == 0;
}
// package private for testing, only used by tests
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
index df9967e..2a1da0d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
@@ -21,21 +21,24 @@ import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.Collections;
import java.util.EnumMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
-import org.apache.commons.lang3.StringUtils;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.response.Cluster;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
+import org.apache.solr.cloud.overseer.ZkWriteCommand;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@@ -60,6 +63,7 @@ import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
@@ -82,27 +86,45 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
public static final String SKIP_NODE_ASSIGNMENT = "skipNodeAssignment";
private final OverseerCollectionMessageHandler ocmh;
+ private final boolean onlyUpdateState;
+ private boolean createdShardHandler;
public AddReplicaCmd(OverseerCollectionMessageHandler ocmh) {
+ this.onlyUpdateState = false;
+ this.ocmh = ocmh;
+ }
+
+ public AddReplicaCmd(OverseerCollectionMessageHandler ocmh, boolean onlyUpdateState) {
+ this.onlyUpdateState = onlyUpdateState;
this.ocmh = ocmh;
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
- Response response = addReplica(state, message, results, null);
- if (response == null) return null;
- return response.asyncFinalRunner;
+ public Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ ShardHandler shardHandler = null;
+ ShardRequestTracker shardRequestTracker = null;
+
+ if (!onlyUpdateState) {
+ final String asyncId = message.getStr(ASYNC);
+ shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
+ shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+ createdShardHandler = true;
+ }
+
+ Response response = addReplica(state, message, shardHandler, shardRequestTracker, results);
+ return response;
}
@SuppressWarnings({"unchecked"})
- Response addReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+ Response addReplica(ClusterState clusterState, ZkNodeProps message, ShardHandler shardHandler,
+ ShardRequestTracker shardRequestTracker, @SuppressWarnings({"rawtypes"})NamedList results)
throws IOException, InterruptedException, KeeperException {
log.info("addReplica() : {}", Utils.toJSONString(message));
String extCollectionName = message.getStr(COLLECTION_PROP);
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
- String shard = message.getStr(SHARD_ID_PROP);
final String collectionName;
if (followAliases) {
@@ -111,19 +133,10 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
collectionName = extCollectionName;
}
- DocCollection coll = clusterState.getCollection(collectionName);
- if (coll == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collectionName + " does not exist");
- }
- if (coll.getSlice(shard) == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
- "Collection: " + collectionName + " shard: " + shard + " does not exist");
- }
-
// nocommit
boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
- final String asyncId = message.getStr(ASYNC);
+
String node = message.getStr(CoreAdminParams.NODE);
String createNodeSetStr = message.getStr(ZkStateReader.CREATE_NODE_SET);
@@ -149,83 +162,87 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
if (node != null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create " + totalReplicas + " replicas if 'node' parameter is specified");
}
- if (message.getStr(CoreAdminParams.CORE_NODE_NAME) != null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create " + totalReplicas + " replicas if 'coreNodeName' parameter is specified");
- }
}
- List<CreateReplica> createReplicas;
- try {
- ocmh.zkStateReader.waitForState(collectionName, 15, TimeUnit.SECONDS, (liveNodes, collectionState) -> {
- if (collectionState == null) {
- return false;
- }
- if (collectionState.getSlices() == null) {
- return false;
- }
- return true;
- });
- } catch (TimeoutException e) {
- log.warn("Could not find collection with populated shards in clusterstate \n{}", clusterState);
- }
- clusterState = ocmh.zkStateReader.getClusterState();
+ List<CreateReplica> createReplicas = new ArrayList<>();
+
+ DocCollection collection = clusterState.getCollection(collectionName);
+ List<ReplicaPosition> positions = buildReplicaPositions(ocmh.cloudManager, clusterState, collection, message, replicaTypesVsCount);
+ for (ReplicaPosition replicaPosition : positions) {
+ clusterState = new CollectionMutator(ocmh.cloudManager).modifyCollection(clusterState, message);
+ collection = clusterState.getCollection(collectionName);
+ CreateReplica cr = assignReplicaDetails(collection, message, replicaPosition);
+
+ message = message.plus(ZkStateReader.BASE_URL_PROP, ocmh.zkStateReader.getBaseUrlForNodeName(replicaPosition.node));
- ClusterState finalClusterState = clusterState;
- createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
- .stream()
- .map(replicaPosition -> assignReplicaDetails(ocmh.cloudManager, finalClusterState, message, replicaPosition))
- .collect(Collectors.toList());
+ message = message.plus(NODE_NAME_PROP, replicaPosition.node);
+ message = message.plus(ZkStateReader.REPLICA_TYPE, cr.replicaType.name());
- if (createReplicas.size() != totalReplicas) {
- throw new IllegalStateException("Did not get enough positions to cover new replicas");
+
+ clusterState = new SliceMutator(ocmh.cloudManager).addReplica(clusterState, message);
+ createReplicas.add(cr);
+
+ // message.getProperties().put("node_name", cr.node)
}
- ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
- ZkStateReader zkStateReader = ocmh.zkStateReader;
+// createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collection, message, replicaTypesVsCount)
+// .stream()
+// .map(replicaPosition -> assignReplicaDetails(collection, message, replicaPosition))
+// .collect(Collectors.toList());
+
- final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
for (CreateReplica createReplica : createReplicas) {
- assert createReplica.coreName != null;
- ModifiableSolrParams params = getReplicaParams(clusterState, message, results, collectionName, coll, skipCreateReplicaInClusterState, shardHandler, createReplica);
- shardRequestTracker.sendShardRequest(createReplica.node, params, shardHandler);
- }
- try {
- shardRequestTracker.processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica");
- } catch (Exception e) {
- ParWork.propagateInterrupt(e);
- return null;
+ ModifiableSolrParams params = getReplicaParams(collection, message, results, skipCreateReplicaInClusterState, shardHandler, createReplica);
+
+ log.info("create replica {} params={}", createReplica, params);
+ if (!onlyUpdateState) {
+ shardRequestTracker.sendShardRequest(createReplica.node, params, shardHandler);
+ }
}
- if (onComplete != null) onComplete.run();
+ Response response = new Response();
- if (onComplete != null) onComplete.run();
+ if (!onlyUpdateState) {
+ response.responseProps = createReplicas.stream().map(
+ createReplica -> new ZkNodeProps(ZkStateReader.COLLECTION_PROP, createReplica.collectionName, ZkStateReader.SHARD_ID_PROP, createReplica.sliceName, ZkStateReader.CORE_NAME_PROP,
+ createReplica.coreName, ZkStateReader.NODE_NAME_PROP, createReplica.node)).collect(Collectors.toList());
+ response.results = results;
+
+ ZkNodeProps finalMessage = message;
+ response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+ @Override
+ public Response call() {
+ if (!onlyUpdateState && createdShardHandler) {
+ try {
+ log.info("Processs responses");
+ shardRequestTracker.processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica");
+ } catch (Exception e) {
+ ParWork.propagateInterrupt(e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+ }
+
+ String asyncId = finalMessage.getStr(ASYNC);
+ for (CreateReplica createReplica : createReplicas) {
+ waitForActiveReplica(createReplica.sliceName, collectionName, asyncId, ocmh.zkStateReader, createReplicas);
+ }
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ return response;
+ }
+ };
- Response response = new Response();
- response.responseProps = createReplicas.stream()
- .map(createReplica -> new ZkNodeProps(
- ZkStateReader.COLLECTION_PROP, createReplica.collectionName,
- ZkStateReader.SHARD_ID_PROP, createReplica.sliceName,
- ZkStateReader.CORE_NAME_PROP, createReplica.coreName,
- ZkStateReader.NODE_NAME_PROP, createReplica.node
- ))
- .collect(Collectors.toList());
- if (results.get("failure") == null && results.get("exception") == null) {
- response.asyncFinalRunner = new Runnable() {
- @Override
- public void run() {
- waitForActiveReplica(shard, collectionName, asyncId, zkStateReader, createReplicas);
- }
- };
}
+ response.clusterState = clusterState;
+
return response;
}
private void waitForActiveReplica(String shard, String collectionName, String asyncId, ZkStateReader zkStateReader, List<CreateReplica> createReplicas) {
- Set<String> coreNodeNames = ConcurrentHashMap.newKeySet(createReplicas.size());
+ Set<String> coreNames = new HashSet<>(createReplicas.size());
for (CreateReplica replica : createReplicas) {
- coreNodeNames.add(ocmh.waitForCoreNodeName(zkStateReader, collectionName, replica.node, replica.coreName));
+ coreNames.add(replica.coreName);
}
try {
zkStateReader.waitForState(collectionName, 60, TimeUnit.SECONDS, (liveNodes, collectionState) -> {
@@ -240,14 +257,14 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
List<Replica> replicas = collectionState.getReplicas();
int found = 0;
- for (String name : coreNodeNames) {
+ for (String name : coreNames) {
for (Replica replica : replicas) {
- if (replica.getName().equals(name) && replica.getState().equals(Replica.State.ACTIVE) && liveNodes.contains(replica.getNodeName())) {
+ if (replica.getName().equals(name) && replica.getState().equals(Replica.State.ACTIVE)) {
found++;
}
}
}
- if (found == coreNodeNames.size()) {
+ if (found == coreNames.size()) {
return true;
}
@@ -261,50 +278,43 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
- private ModifiableSolrParams getReplicaParams(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, String collectionName, DocCollection coll, boolean skipCreateReplicaInClusterState, ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException, KeeperException {
- if (coll.getStr(WITH_COLLECTION) != null) {
- String withCollectionName = coll.getStr(WITH_COLLECTION);
- DocCollection withCollection = clusterState.getCollection(withCollectionName);
- if (withCollection.getActiveSlices().size() > 1) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The `withCollection` must have only one shard, found: " + withCollection.getActiveSlices().size());
- }
- String withCollectionShard = withCollection.getActiveSlices().iterator().next().getName();
-
- List<Replica> replicas = withCollection.getReplicas(createReplica.node);
- if (replicas == null || replicas.isEmpty()) {
- // create a replica of withCollection on the identified node before proceeding further
- ZkNodeProps props = new ZkNodeProps(
- Overseer.QUEUE_OPERATION, ADDREPLICA.toString(),
- ZkStateReader.COLLECTION_PROP, withCollectionName,
- ZkStateReader.SHARD_ID_PROP, withCollectionShard,
- "node", createReplica.node,
- // since we already computed node assignments (which include assigning a node for this withCollection replica) we want to skip the assignment step
- SKIP_NODE_ASSIGNMENT, "true",
- CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.TRUE.toString()); // set to true because we want `withCollection` to be ready after this collection is created
- addReplica(clusterState, props, results, null);
- }
- }
+ private ModifiableSolrParams getReplicaParams( DocCollection collection, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results,
+ boolean skipCreateReplicaInClusterState,
+ ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException, KeeperException {
+
+ // nocommit
+// if (collection.getStr(WITH_COLLECTION) != null) {
+// String withCollectionName = collection.getStr(WITH_COLLECTION);
+// DocCollection withCollection = clusterState.getCollection(withCollectionName);
+// if (withCollection.getActiveSlices().size() > 1) {
+// throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The `withCollection` must have only one shard, found: " + withCollection.getActiveSlices().size());
+// }
+// String withCollectionShard = withCollection.getActiveSlices().iterator().next().getName();
+//
+// List<Replica> replicas = withCollection.getReplicas(createReplica.node);
+// if (replicas == null || replicas.isEmpty()) {
+// // create a replica of withCollection on the identified node before proceeding further
+// ZkNodeProps props = new ZkNodeProps(
+// Overseer.QUEUE_OPERATION, ADDREPLICA.toString(),
+// ZkStateReader.COLLECTION_PROP, withCollectionName,
+// ZkStateReader.SHARD_ID_PROP, withCollectionShard,
+// CORE_NAME_PROP, createReplica.coreName,
+// "node", createReplica.node,
+// // since we already computed node assignments (which include assigning a node for this withCollection replica) we want to skip the assignment step
+// SKIP_NODE_ASSIGNMENT, "true",
+// CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.TRUE.toString()); // set to true because we want `withCollection` to be ready after this collection is created
+// addReplica(clusterState, props, results);
+// }
+// }
ModifiableSolrParams params = new ModifiableSolrParams();
ZkStateReader zkStateReader = ocmh.zkStateReader;
-
+ String collectionName = collection.getName();
ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(), ZkStateReader.COLLECTION_PROP, collectionName, ZkStateReader.SHARD_ID_PROP, createReplica.sliceName,
ZkStateReader.CORE_NAME_PROP, createReplica.coreName, ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP,
zkStateReader.getBaseUrlForNodeName(createReplica.node), ZkStateReader.NODE_NAME_PROP, createReplica.node, ZkStateReader.REPLICA_TYPE, createReplica.replicaType.name());
- if (createReplica.coreNodeName != null) {
- props = props.plus(ZkStateReader.CORE_NODE_NAME_PROP, createReplica.coreNodeName);
- }
- if (!skipCreateReplicaInClusterState) {
- try {
- ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
- } catch (Exception e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Exception updating Overseer state queue", e);
- }
- }
+
String coreUrl = ZkCoreNodeProps.getCoreUrl(props.getStr(ZkStateReader.BASE_URL_PROP), createReplica.coreName);
- ;
- params.set(CoreAdminParams.CORE_NODE_NAME, ocmh.waitToSeeReplicasInState(collectionName, Collections.singletonList(coreUrl), false).get(coreUrl).getName());
String configName = zkStateReader.readConfigName(collectionName);
String routeKey = message.getStr(ShardParams._ROUTE_);
@@ -317,10 +327,12 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
params.set(COLL_CONF, configName);
params.set(CoreAdminParams.COLLECTION, collectionName);
params.set(CoreAdminParams.REPLICA_TYPE, createReplica.replicaType.name());
+
+ log.info("Creating SolrCore with name={}", createReplica.coreName);
if (createReplica.sliceName != null) {
params.set(CoreAdminParams.SHARD, createReplica.sliceName);
} else if (routeKey != null) {
- Collection<Slice> slices = coll.getRouter().getSearchSlicesSingle(routeKey, null, coll);
+ Collection<Slice> slices = collection.getRouter().getSearchSlicesSingle(routeKey, null, collection);
if (slices.isEmpty()) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No active shard serving _route_=" + routeKey + " found");
} else {
@@ -338,71 +350,41 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
if (instanceDir != null) {
params.set(CoreAdminParams.INSTANCE_DIR, instanceDir);
}
- if (createReplica.coreNodeName != null) {
- params.set(CoreAdminParams.CORE_NODE_NAME, createReplica.coreNodeName);
- }
+
ocmh.addPropertyParams(message, params);
return params;
}
- public static CreateReplica assignReplicaDetails(SolrCloudManager cloudManager, ClusterState clusterState,
+ public static CreateReplica assignReplicaDetails(DocCollection coll,
ZkNodeProps message, ReplicaPosition replicaPosition) {
+
+ log.info("assignReplicaDetails {} {} {}", message, replicaPosition, coll);
+
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
String collection = message.getStr(COLLECTION_PROP);
String node = replicaPosition.node;
String shard = message.getStr(SHARD_ID_PROP);
- String coreName = message.getStr(CoreAdminParams.NAME);
- String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
+ String coreName = message.getStr(CoreAdminParams.CORE);
Replica.Type replicaType = replicaPosition.type;
- if (StringUtils.isBlank(coreName)) {
- coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
- }
+ log.info("Node Identified {} for creating new replica (core={}) of shard {} for collection {} currentReplicaCount {}", node, coreName, shard, collection, coll.getReplicas().size());
- log.info("Node Identified {} for creating new replica of shard {} for collection {}", node, shard, collection);
- if (!clusterState.liveNodesContain(node)) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
- }
- DocCollection coll = clusterState.getCollection(collection);
if (coreName == null) {
- coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), coll, shard, replicaType);
- } else if (!skipCreateReplicaInClusterState) {
- //Validate that the core name is unique in that collection
- while (true) {
- try {
- clusterState = cloudManager.getClusterStateProvider().getClusterState();
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
- }
- coll = clusterState.getCollection(collection);
- for (Slice slice : coll.getSlices()) {
- for (Replica replica : slice.getReplicas()) {
- String replicaCoreName = replica.getStr(CORE_NAME_PROP);
- if (coreName.equals(replicaCoreName)) {
- log.info("Another replica with the same core name already exists" +
- " for this collection");
- coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), coll, shard, replicaType);
- continue;
- }
- }
- }
- break;
- }
+ coreName = Assign.buildSolrCoreName(coll, coll.getName(), shard, replicaType);
}
- log.info("Returning CreateReplica command.");
+ log.info("Returning CreateReplica command coreName={}", coreName);
- return new CreateReplica(collection, shard, node, replicaType, coreName, coreNodeName);
+ return new CreateReplica(collection, shard, node, replicaType, coreName);
}
- public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
- String collectionName, ZkNodeProps message,
+ public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState, DocCollection collection,
+ ZkNodeProps message,
EnumMap<Replica.Type, Integer> replicaTypeVsCount) throws IOException, InterruptedException {
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
boolean skipNodeAssignment = message.getBool(SKIP_NODE_ASSIGNMENT, false);
String sliceName = message.getStr(SHARD_ID_PROP);
- DocCollection collection = clusterState.getCollection(collectionName);
int numNrtReplicas = replicaTypeVsCount.get(Replica.Type.NRT);
int numPullReplicas = replicaTypeVsCount.get(Replica.Type.PULL);
@@ -421,7 +403,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
List<ReplicaPosition> positions = null;
if (!skipCreateReplicaInClusterState && !skipNodeAssignment) {
- positions = Assign.getNodesForNewReplicas(clusterState, collection.getName(), sliceName, numNrtReplicas,
+ positions = Assign.getNodesForNewReplicas(clusterState, collection, sliceName, numNrtReplicas,
numTlogReplicas, numPullReplicas, createNodeSetStr, cloudManager);
}
@@ -457,21 +439,23 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
public final String node;
public final Replica.Type replicaType;
public String coreName;
- public String coreNodeName;
- CreateReplica(String collectionName, String sliceName, String node, Replica.Type replicaType, String coreName, String coreNodeName) {
+ CreateReplica(String collectionName, String sliceName, String node, Replica.Type replicaType, String coreName) {
this.collectionName = collectionName;
this.sliceName = sliceName;
this.node = node;
this.replicaType = replicaType;
this.coreName = coreName;
- this.coreNodeName = coreNodeName;
}
}
public static class Response {
List<ZkNodeProps> responseProps;
- Runnable asyncFinalRunner;
+ OverseerCollectionMessageHandler.Finalize asyncFinalRunner;
+
+ NamedList results;
+
+ ClusterState clusterState;
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
index ade7396..349ba3d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
@@ -92,7 +92,7 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
int numShards = BaseCloudSolrClient.getShardNames(zkProps).size();
- BaseCloudSolrClient.waitForActiveCollection(ocmh.zkStateReader, createCollName, 60, TimeUnit.SECONDS, numShards, numShards * BaseCloudSolrClient.getTotalReplicas(zkProps));
+ ocmh.zkStateReader.waitForActiveCollection(createCollName, 60, TimeUnit.SECONDS, numShards, numShards * BaseCloudSolrClient.getTotalReplicas(zkProps));
CollectionProperties collectionProperties = new CollectionProperties(ocmh.zkStateReader);
collectionProperties.setCollectionProperty(createCollName,ROUTED_ALIAS_NAME_CORE_PROP,aliasName);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
index dc60668..4aa567f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
@@ -57,14 +57,6 @@ public class Assign {
private static AtomicInteger REPLICA_CNT = new AtomicInteger(0);
- public static String assignCoreNodeName(DistribStateManager stateManager, DocCollection collection) {
- // for backward compatibility;
- int defaultValue = defaultCounterValue(collection, "");
- String coreNodeName = "core_node" + defaultValue;
-
- return coreNodeName;
- }
-
/**
* Assign a new unique id up to slices count - then add replicas evenly.
*
@@ -81,13 +73,13 @@ public class Assign {
// TODO: now that we create shards ahead of time, is this code needed? Esp since hash ranges aren't assigned when creating via this method?
if (sliceMap == null) {
- return "shard1";
+ return "s";
}
List<String> shardIdNames = new ArrayList<>(sliceMap.keySet());
if (shardIdNames.size() < numShards) {
- return "shard" + (shardIdNames.size() + 1);
+ return "s" + (shardIdNames.size() + 1);
}
// TODO: don't need to sort to find shard with fewest replicas!
@@ -109,42 +101,33 @@ public class Assign {
return returnShardId;
}
- public static String buildSolrCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
+ private static String buildSolrCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
// TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type
- return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
+ return String.format(Locale.ROOT, "%s_%s_r_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
}
- public static int defaultCounterValue(DocCollection collection, String shard) {
- int defaultValue;
- if (collection.getSlice(shard) != null && collection.getSlice(shard).getReplicas().isEmpty()) {
- return REPLICA_CNT.incrementAndGet();
- } else {
- defaultValue = collection.getReplicas().size() + REPLICA_CNT.incrementAndGet() * 2;
- }
+ public static int defaultCounterValue(DocCollection coll, String shard) {
- return defaultValue;
- }
+ if (coll == null) {
+ throw new NullPointerException("DocCollection cannot be null");
+ }
- public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
+ if (coll.getSlice(shard) == null) {
+ return 1;
+ }
- int defaultValue = defaultCounterValue(collection, shard);
- String coreName = buildSolrCoreName(collection.getName(), shard, type, defaultValue);
+ if (coll.getSlice(shard).getReplicas() == null) {
+ return 1;
+ }
- return coreName;
+ return coll.getSlice(shard).getReplicas().size() + 1;
}
- public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
- return buildSolrCoreName(stateManager, collection, shard, type, false);
- }
+ public static String buildSolrCoreName(DocCollection coll, String collectionName, String shard, Replica.Type type) {
+ int defaultValue = defaultCounterValue(coll, shard);
+ String coreName = buildSolrCoreName(collectionName, shard, type, defaultValue);
- private static boolean existCoreName(String coreName, Slice slice) {
- if (slice == null) return false;
- for (Replica replica : slice.getReplicas()) {
- if (coreName.equals(replica.getStr(CORE_NAME_PROP))) {
- return true;
- }
- }
- return false;
+ return coreName;
}
public static List<String> getLiveOrLiveAndCreateNodeSetList(final Collection<String> liveNodes, final ZkNodeProps message, final Random random) {
@@ -188,13 +171,13 @@ public class Assign {
// Gets a list of candidate nodes to put the required replica(s) on. Throws errors if not enough replicas
// could be created on live nodes given maxShardsPerNode, Replication factor (if from createShard) etc.
@SuppressWarnings({"unchecked"})
- public static List<ReplicaPosition> getNodesForNewReplicas(ClusterState clusterState, String collectionName,
+ public static List<ReplicaPosition> getNodesForNewReplicas(ClusterState clusterState, DocCollection collection,
String shard, int nrtReplicas, int tlogReplicas, int pullReplicas,
Object createNodeSet, SolrCloudManager cloudManager) throws IOException, InterruptedException, AssignmentException {
log.debug("getNodesForNewReplicas() shard: {} , nrtReplicas : {} , tlogReplicas: {} , pullReplicas: {} , createNodeSet {}"
, shard, nrtReplicas, tlogReplicas, pullReplicas, createNodeSet);
- DocCollection coll = clusterState.getCollection(collectionName);
- int maxShardsPerNode = coll.getMaxShardsPerNode() == -1 ? Integer.MAX_VALUE : coll.getMaxShardsPerNode();
+
+ //int maxShardsPerNode = collection.getMaxShardsPerNode() == -1 ? Integer.MAX_VALUE : collection.getMaxShardsPerNode();
List<String> createNodeList = null;
if (createNodeSet instanceof List) {
@@ -203,23 +186,9 @@ public class Assign {
// deduplicate
createNodeList = createNodeSet == null ? null : new ArrayList<>(new LinkedHashSet<>(StrUtils.splitSmart((String) createNodeSet, ",", true)));
}
+ String collectionName = collection.getName();
+ HashMap<String, ReplicaCount> nodeNameVsShardCount = getNodeNameVsShardCount(collectionName, clusterState, cloudManager.getClusterStateProvider().getClusterState().getLiveNodes(), createNodeList);
- HashMap<String, ReplicaCount> nodeNameVsShardCount = getNodeNameVsShardCount(collectionName, clusterState, createNodeList);
-
- if (createNodeList == null) { // We only care if we haven't been told to put new replicas on specific nodes.
- long availableSlots = 0;
- for (Map.Entry<String, ReplicaCount> ent : nodeNameVsShardCount.entrySet()) {
- //ADDREPLICA can put more than maxShardsPerNode on an instance, so this test is necessary.
- if (maxShardsPerNode > ent.getValue().thisCollectionNodes) {
- availableSlots += (maxShardsPerNode - ent.getValue().thisCollectionNodes);
- }
- }
- if (availableSlots < nrtReplicas + tlogReplicas + pullReplicas) {
- throw new AssignmentException(
- String.format(Locale.ROOT, "Cannot create %d new replicas for collection %s given the current number of eligible live nodes %d and a maxShardsPerNode of %d",
- nrtReplicas, collectionName, nodeNameVsShardCount.size(), maxShardsPerNode));
- }
- }
AssignRequest assignRequest = new AssignRequestBuilder()
.forCollection(collectionName)
@@ -230,16 +199,15 @@ public class Assign {
.onNodes(createNodeList)
.build();
AssignStrategyFactory assignStrategyFactory = new AssignStrategyFactory(cloudManager);
- AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, coll);
+ AssignStrategy assignStrategy = assignStrategyFactory.create();
return assignStrategy.assign(cloudManager, assignRequest);
}
- static HashMap<String, ReplicaCount> getNodeNameVsShardCount(String collectionName,
- ClusterState clusterState, List<String> createNodeList) {
- Set<String> nodes = clusterState.getLiveNodes();
+ static HashMap<String, ReplicaCount> getNodeNameVsShardCount(String collectionName, ClusterState clusterState,
+ Set<String> liveNodes, List<String> createNodeList) {
- List<String> nodeList = new ArrayList<>(nodes.size());
- nodeList.addAll(nodes);
+ List<String> nodeList = new ArrayList<>(liveNodes.size());
+ nodeList.addAll(liveNodes);
if (createNodeList != null) nodeList.retainAll(createNodeList);
HashMap<String, ReplicaCount> nodeNameVsShardCount = new HashMap<>(nodeList.size());
@@ -255,8 +223,6 @@ public class Assign {
return nodeNameVsShardCount;
}
- DocCollection coll = clusterState.getCollection(collectionName);
- int maxShardsPerNode = coll.getMaxShardsPerNode() == -1 ? Integer.MAX_VALUE : coll.getMaxShardsPerNode();
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
if (entry.getValue() == null) continue;
@@ -379,7 +345,8 @@ public class Assign {
ClusterState clusterState = solrCloudManager.getClusterStateProvider().getClusterState();
List<String> nodeList = assignRequest.nodes;
- HashMap<String, Assign.ReplicaCount> nodeNameVsShardCount = Assign.getNodeNameVsShardCount(assignRequest.collectionName, clusterState, assignRequest.nodes);
+ HashMap<String, Assign.ReplicaCount> nodeNameVsShardCount = Assign.getNodeNameVsShardCount(assignRequest.collectionName,
+ clusterState, solrCloudManager.getClusterStateProvider().getLiveNodes(),assignRequest.nodes);
if (nodeList == null || nodeList.isEmpty()) {
ArrayList<Assign.ReplicaCount> sortedNodeList = new ArrayList<>(nodeNameVsShardCount.values());
sortedNodeList.sort(Comparator.comparingInt(Assign.ReplicaCount::weight));
@@ -409,7 +376,7 @@ public class Assign {
this.solrCloudManager = solrCloudManager;
}
- public AssignStrategy create(ClusterState clusterState, DocCollection collection) throws IOException, InterruptedException {
+ public AssignStrategy create() throws IOException, InterruptedException {
return new LegacyAssignStrategy();
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
index eef735d..666bd71 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
@@ -29,6 +29,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.Optional;
import java.util.Properties;
+import java.util.stream.Collectors;
import org.apache.lucene.util.Version;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
@@ -41,6 +42,7 @@ import org.apache.solr.common.cloud.Replica.State;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
@@ -66,7 +68,7 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String extCollectionName = message.getStr(COLLECTION_PROP);
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
@@ -135,7 +137,14 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
backupMgr.downloadCollectionProperties(location, backupName, collectionName);
log.info("Completed backing up ZK data for backupName={}", backupName);
- return null;
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ response.results = results;
+
+ // nocommit - we don't change this for this cmd, we should be able to indicate that to caller
+ response.clusterState = clusterState;
+
+ return response;
}
private Replica selectReplicaWithSnapshot(CollectionSnapshotMetaData snapshotMeta, Slice slice) {
@@ -215,20 +224,18 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
- String coreName = replica.getStr(CORE_NAME_PROP);
-
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.BACKUPCORE.toString());
params.set(NAME, slice.getName());
params.set(CoreAdminParams.BACKUP_REPOSITORY, repoName);
params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString()); // note: index dir will be here then the "snapshot." + slice name
- params.set(CORE_NAME_PROP, coreName);
+ params.set(CORE_NAME_PROP, replica.getName());
if (snapshotMeta.isPresent()) {
params.set(CoreAdminParams.COMMIT_NAME, snapshotMeta.get().getName());
}
shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
- log.debug("Sent backup request to core={} for backupName={}", coreName, backupName);
+ log.debug("Sent backup request to core={} for backupName={}", replica.getName(), backupName);
}
log.debug("Sent backup requests to all shard leaders for backupName={}", backupName);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
index d39f4b8..1cb9771 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
@@ -52,7 +52,7 @@ public class CreateAliasCmd extends AliasCmd {
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
final String aliasName = message.getStr(CommonParams.NAME);
ZkStateReader zkStateReader = ocmh.zkStateReader;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index 4ae8fb6..7f133bd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -29,9 +29,9 @@ import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Properties;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
@@ -41,6 +41,8 @@ import org.apache.solr.client.solrj.impl.BaseCloudSolrClient;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@@ -86,7 +88,6 @@ import static org.apache.solr.common.params.CollectionAdminParams.COLOCATED_WITH
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MODIFYCOLLECTION;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
-import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.util.StrUtils.formatString;
@@ -99,6 +100,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
private final SolrCloudManager cloudManager;
public CreateCollectionCmd(OverseerCollectionMessageHandler ocmh, CoreContainer cc, SolrCloudManager cloudManager) {
+ log.info("create CreateCollectionCmd");
this.ocmh = ocmh;
this.stateManager = ocmh.cloudManager.getDistribStateManager();
this.timeSource = ocmh.cloudManager.getTimeSource();
@@ -108,13 +110,17 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
@Override
@SuppressWarnings({"unchecked"})
- public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ log.info("run call for CreateCollectionCmd {}", message);
if (ocmh.zkStateReader.aliasesManager != null) { // not a mock ZkStateReader
- ocmh.zkStateReader.aliasesManager.update();
+ ocmh.zkStateReader.aliasesManager.update(); // nocommit - hate this
}
+ final String async = message.getStr(ASYNC);
+ Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
+ List<ReplicaPosition> replicaPositions = null;
final Aliases aliases = ocmh.zkStateReader.getAliases();
final String collectionName = message.getStr(NAME);
- final boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
+ final boolean waitForFinalState = false;
final String alias = message.getStr(ALIAS, collectionName);
log.info("Create collection {}", collectionName);
if (clusterState.hasCollection(collectionName)) {
@@ -124,6 +130,9 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "collection alias already exists: " + collectionName);
}
+ final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(async, message.getStr("operation"));
+ ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
String withCollection = message.getStr(CollectionAdminParams.WITH_COLLECTION);
String withCollectionShard = null;
if (withCollection != null) {
@@ -155,8 +164,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
try {
- final String async = message.getStr(ASYNC);
-
Map<String,String> collectionParams = new HashMap<>();
Map<String,Object> collectionProps = message.getProperties();
for (Map.Entry<String,Object> entry : collectionProps.entrySet()) {
@@ -174,55 +181,12 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
OverseerCollectionMessageHandler.createConfNode(stateManager, configName, collectionName);
- // TODO need to make this makePath calls efficient and not use zkSolrClient#makePath
- for (String shardName : shardNames) {
- try {
- stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/" + shardName, null, CreateMode.PERSISTENT, false);
- // stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect", null, CreateMode.PERSISTENT, false);
- stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName, null, CreateMode.PERSISTENT, false);
- stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName + "/election", null, CreateMode.PERSISTENT, false);
- stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leaders/" + shardName, null, CreateMode.PERSISTENT, false);
- stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/terms/" + shardName, ZkStateReader.emptyJson, CreateMode.PERSISTENT, false);
- } catch (AlreadyExistsException e) {
- // okay
- }
- }
-
- if (log.isDebugEnabled()) log.debug("Offer create operation to Overseer queue");
- ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
-
- // nocommit
- // wait for a while until we see the collection
-
- ocmh.zkStateReader.waitForState(collectionName, 10, TimeUnit.SECONDS, (n, c) -> c != null);
-
- // refresh cluster state
- clusterState = ocmh.zkStateReader.getClusterState();
-
- List<ReplicaPosition> replicaPositions = null;
- // try {
- // replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState,
- // clusterState.getCollection(collectionName), message, shardNames, sessionWrapper);
- // } catch (Exception e) {
- // ParWork.propegateInterrupt(e);
- // SolrException exp = new SolrException(ErrorCode.SERVER_ERROR, "call(ClusterState=" + clusterState + ", ZkNodeProps=" + message + ", NamedList=" + results + ")", e);
- // try {
- // ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
- // new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
- // // unwrap the exception
- // } catch (Exception e1) {
- // ParWork.propegateInterrupt(e1);
- // exp.addSuppressed(e1);
- // }
- // throw exp;
- // }
-
- DocCollection docCollection = clusterState.getCollection(collectionName);
+ DocCollection docCollection = buildDocCollection(stateManager, message, true);
+ clusterState = clusterState.copyWith(collectionName, docCollection);
try {
- replicaPositions = buildReplicaPositions(cloudManager, clusterState, docCollection, message, shardNames);
+ replicaPositions = buildReplicaPositions(cloudManager, message, shardNames);
} catch (Exception e) {
- ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
- new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
+ log.error("", e);
// unwrap the exception
throw new SolrException(ErrorCode.BAD_REQUEST, e.getMessage(), e.getCause());
}
@@ -232,29 +196,27 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
// throw new SolrException(ErrorCode.SERVER_ERROR, "No positions found to place replicas " + replicaPositions);
// }
- final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(async, message.getStr("operation"));
if (log.isDebugEnabled()) {
- log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , message : {2}", collectionName, shardNames, message));
+ log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} replicaCount {2}, message : {3}", collectionName, shardNames, replicaPositions.size(), message));
}
- Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
- ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
for (ReplicaPosition replicaPosition : replicaPositions) {
String nodeName = replicaPosition.node;
if (withCollection != null) {
// check that we have a replica of `withCollection` on this node and if not, create one
- DocCollection collection = clusterState.getCollection(withCollection);
- List<Replica> replicas = collection.getReplicas(nodeName);
+ DocCollection wcollection = clusterState.getCollection(withCollection);
+ List<Replica> replicas = wcollection.getReplicas(nodeName);
if (replicas == null || replicas.isEmpty()) {
ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toString(), ZkStateReader.COLLECTION_PROP, withCollection, ZkStateReader.SHARD_ID_PROP, withCollectionShard,
"node", nodeName, CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.TRUE.toString()); // set to true because we want `withCollection` to be ready after this collection is created
- // TODO: prob want too look into this being parallel or always async
- new AddReplicaCmd(ocmh).call(clusterState, props, results);
- clusterState = zkStateReader.getClusterState(); // refresh
+
+ new AddReplicaCmd(ocmh, true).call(clusterState, props, results);
+ clusterState = new SliceMutator(cloudManager).addReplica(clusterState, props);
}
}
-
- String coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), docCollection, replicaPosition.shard, replicaPosition.type, true);
+ DocCollection coll = clusterState.getCollectionOrNull(collectionName);
+ String coreName = Assign.buildSolrCoreName(coll, collectionName, replicaPosition.shard, replicaPosition.type);
log.info(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}", coreName, replicaPosition.shard, collectionName, nodeName));
String baseUrl = zkStateReader.getBaseUrlForNodeName(nodeName);
@@ -264,14 +226,17 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
log.info("Base url for replica={}", baseUrl);
ZkNodeProps props = new ZkNodeProps();
- props.getProperties().putAll(message.getProperties());
+ //props.getProperties().putAll(message.getProperties());
ZkNodeProps addReplicaProps = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toString(), ZkStateReader.COLLECTION_PROP, collectionName, ZkStateReader.SHARD_ID_PROP,
replicaPosition.shard, ZkStateReader.CORE_NAME_PROP, coreName, ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP, baseUrl, ZkStateReader.NODE_NAME_PROP,
- nodeName, ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(), ZkStateReader.NUM_SHARDS_PROP, message.getStr(ZkStateReader.NUM_SHARDS_PROP), "shards", message.getStr("shards"),
- CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
- props.getProperties().putAll(addReplicaProps.getProperties());
+ nodeName, "node", nodeName, ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(), ZkStateReader.NUM_SHARDS_PROP, message.getStr(ZkStateReader.NUM_SHARDS_PROP), "shards",
+ message.getStr("shards"), CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState)); props.getProperties().putAll(addReplicaProps.getProperties());
if (log.isDebugEnabled()) log.debug("Sending state update to populate clusterstate with new replica {}", props);
- ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
+
+ clusterState = new AddReplicaCmd(ocmh, true).call(clusterState, props, results).clusterState;
+ log.info("CreateCollectionCmd after add replica clusterstate={}", clusterState);
+
+ //clusterState = new SliceMutator(cloudManager).addReplica(clusterState, props);
// Need to create new params for each request
ModifiableSolrParams params = new ModifiableSolrParams();
@@ -282,6 +247,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
params.set(CoreAdminParams.COLLECTION, collectionName);
params.set(CoreAdminParams.SHARD, replicaPosition.shard);
params.set(ZkStateReader.NUM_SHARDS_PROP, shardNames.size());
+ params.set(ZkStateReader.NODE_NAME_PROP, nodeName);
params.set(CoreAdminParams.NEW_COLLECTION, "true");
params.set(CoreAdminParams.REPLICA_TYPE, replicaPosition.type.name());
@@ -302,70 +268,19 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
coresToCreate.put(coreName, sreq);
}
-
- // wait for all replica entries to be created
- Map<String,Replica> replicas = new ConcurrentHashMap<>();
-
- zkStateReader.waitForState(collectionName, 30, TimeUnit.SECONDS,
- expectedReplicas(coresToCreate.size(), replicas)); // nocommit - timeout - keep this below containing timeouts - need central timeout stuff
- // TODO what if replicas comes back wrong?
- if (replicas.size() > 0) {
- for (Map.Entry<String,ShardRequest> e : coresToCreate.entrySet()) {
- ShardRequest sreq = e.getValue();
- Replica replica = null;
- for (Replica rep : replicas.values()) {
- log.info("cmp {} {} {} {}", e.getKey(), sreq.shards[0], rep.getCoreName(), rep.getBaseUrl());
- if (rep.getCoreName().equals(e.getKey()) && rep.getBaseUrl().equals(sreq.shards[0])) {
- sreq.params.set(CoreAdminParams.CORE_NODE_NAME, rep.getName());
- replica = rep;
- break;
- }
- }
-
- if (sreq.params.get(CoreAdminParams.CORE_NODE_NAME) == null || replica == null) {
- continue;
- }
-
- log.info("Submit request to shard for for replica={}", sreq.actualShards != null ? Arrays.asList(sreq.actualShards) : "null");
- shardHandler.submit(sreq, sreq.shards[0], sreq.params);
- }
- }
-
- shardRequestTracker.processResponses(results, shardHandler, false, null, Collections.emptySet());
- @SuppressWarnings({"rawtypes"}) boolean failure = results.get("failure") != null && ((SimpleOrderedMap) results.get("failure")).size() > 0;
- if (failure) {
- // Let's cleanup as we hit an exception
- // We shouldn't be passing 'results' here for the cleanup as the response would then contain 'success'
- // element, which may be interpreted by the user as a positive ack
- ocmh.cleanupCollection(collectionName, new NamedList<Object>());
- log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
- throw new SolrException(ErrorCode.BAD_REQUEST, "Underlying core creation failed while creating collection: " + collectionName + "\n" + results);
- } else {
- log.debug("Finished create command on all shards for collection: {}", collectionName);
-
- // Emit a warning about production use of data driven functionality
- boolean defaultConfigSetUsed = message.getStr(COLL_CONF) == null || message.getStr(COLL_CONF).equals(ConfigSetsHandlerApi.DEFAULT_CONFIGSET_NAME);
- if (defaultConfigSetUsed) {
- results.add("warning",
- "Using _default configset. Data driven schema functionality" + " is enabled by default, which is NOT RECOMMENDED for production use. To turn it off:" + " curl http://{host:port}/solr/"
- + collectionName + "/config -d '{\"set-user-property\": {\"update.autoCreateFields\":\"false\"}}'");
- }
- if (async == null) {
- zkStateReader.waitForState(collectionName, 30, TimeUnit.SECONDS, BaseCloudSolrClient.expectedShardsAndActiveReplicas(shardNames.size(), replicaPositions.size()));
- }
+ log.info("Sending create call for {} replicas", coresToCreate.size());
+ for (Map.Entry<String,ShardRequest> e : coresToCreate.entrySet()) {
+ ShardRequest sreq = e.getValue();
+ log.info("Submit request to shard for for replica coreName={} total requests={} shards={}", e.getKey(), coresToCreate.size(),
+ sreq.actualShards != null ? Arrays.asList(sreq.actualShards) : "null");
+ // nocommit - work out parallel / async 100%
+ shardHandler.submit(sreq, sreq.shards[0], sreq.params);
}
// modify the `withCollection` and store this new collection's name with it
if (withCollection != null) {
ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, MODIFYCOLLECTION.toString(), ZkStateReader.COLLECTION_PROP, withCollection, CollectionAdminParams.COLOCATED_WITH, collectionName);
- ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
- try {
- zkStateReader.waitForState(withCollection, 30, TimeUnit.SECONDS, (collectionState) -> collectionName.equals(collectionState.getStr(COLOCATED_WITH)));
- } catch (TimeoutException e) {
- log.warn("Timed out waiting to see the {} property set on collection: {}", COLOCATED_WITH, withCollection);
- // maybe the overseer queue is backed up, we don't want to fail the create request
- // because of this time out, continue
- }
+ clusterState = new CollectionMutator(cloudManager).modifyCollection(clusterState, props);
}
// create an alias pointing to the new collection, if different from the collectionName
@@ -383,16 +298,70 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
log.error("Exception creating collection", ex);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
}
- return null;
+
+ log.info("CreateCollectionCmd clusterstate={}", clusterState);
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ if (results.get("failure") == null && results.get("exception") == null) {
+ List<ReplicaPosition> finalReplicaPositions = replicaPositions;
+ response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+ @Override
+ public AddReplicaCmd.Response call() {
+ try {
+ shardRequestTracker.processResponses(results, shardHandler, false, null, Collections.emptySet());
+ } catch (KeeperException e) {
+ log.error("", e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ } catch (InterruptedException e) {
+ ParWork.propagateInterrupt(e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+ // nocommit - put this in finalizer and finalizer after all calls to allow parallel and forward momentum
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ @SuppressWarnings({"rawtypes"}) boolean failure = results.get("failure") != null && ((SimpleOrderedMap) results.get("failure")).size() > 0;
+ if (failure) {
+ // // Let's cleanup as we hit an exception
+ // // We shouldn't be passing 'results' here for the cleanup as the response would then contain 'success'
+ // // element, which may be interpreted by the user as a positive ack
+ // // nocommit review
+ try {
+ response.clusterState = ocmh.cleanupCollection(collectionName, new NamedList<Object>()).clusterState;
+ } catch (Exception e) {
+ log.error("Exception trying to clean up collection after fail {}", collectionName);
+ }
+ log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
+ //throw new SolrException(ErrorCode.BAD_REQUEST, "Underlying core creation failed while creating collection: " + collectionName + "\n" + results);
+ } else {
+ if (log.isDebugEnabled()) log.debug("Finished create command on all shards for collection: {}", collectionName);
+
+ // Emit a warning about production use of data driven functionality
+ boolean defaultConfigSetUsed = message.getStr(COLL_CONF) == null || message.getStr(COLL_CONF).equals(ConfigSetsHandlerApi.DEFAULT_CONFIGSET_NAME);
+ if (defaultConfigSetUsed) {
+ results.add("warning",
+ "Using _default configset. Data driven schema functionality" + " is enabled by default, which is NOT RECOMMENDED for production use. To turn it off:" + " curl http://{host:port}/solr/"
+ + collectionName + "/config -d '{\"set-user-property\": {\"update.autoCreateFields\":\"false\"}}'");
+ }
+
+ }
+
+
+ zkStateReader.waitForActiveCollection(collectionName, 10, TimeUnit.SECONDS, shardNames.size(), finalReplicaPositions.size());
+ return response;
+ }
+ };
+ }
+ response.clusterState = clusterState;
+ return response;
}
- public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
- DocCollection docCollection,
+ public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager,
ZkNodeProps message,
List<String> shardNames) throws IOException, InterruptedException, Assign.AssignmentException {
if (log.isDebugEnabled()) {
- // nocommit
- // log.debug("buildReplicaPositions(SolrCloudManager cloudManager={}, ClusterState clusterState={}, DocCollection docCollection={}, ZkNodeProps message={}, List<String> shardNames={}, AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper={}) - start", cloudManager, clusterState, docCollection, message, shardNames, sessionWrapper);
+ log.debug("buildReplicaPositions(SolrCloudManager cloudManager={}, ZkNodeProps message={}, List<String> shardNames={}) - start",
+ cloudManager, message, shardNames);
}
final String collectionName = message.getStr(NAME);
@@ -418,7 +387,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
throw new SolrException(ErrorCode.SERVER_ERROR, e);
}
if (nodeList.isEmpty()) {
- log.warn("It is unusual to create a collection ("+collectionName+") without cores. liveNodes={} message={}", clusterState.getLiveNodes(), message);
+ log.warn("It is unusual to create a collection ("+collectionName+") without cores. message={}", message);
replicaPositions = new ArrayList<>();
} else {
@@ -433,24 +402,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
+ "). It's unusual to run two replica of the same slice on the same Solr-instance.");
}
- int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
- Integer.MAX_VALUE :
- maxShardsPerNode * nodeList.size();
- int requestedShardsToCreate = numSlices * totalNumReplicas;
- if (maxShardsAllowedToCreate < requestedShardsToCreate) {
- String msg = "Cannot create collection " + collectionName + ". Value of "
- + MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
- + ", and the number of nodes currently live or live and part of your "+ZkStateReader.CREATE_NODE_SET+" is " + nodeList.size()
- + ". This allows a maximum of " + maxShardsAllowedToCreate
- + " to be created. Value of " + ZkStateReader.NUM_SHARDS_PROP + " is " + numSlices
- + ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
- + ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
- + " and value of " + PULL_REPLICAS + " is " + numPullReplicas
- + ". This requires " + requestedShardsToCreate
- + " shards to be created (higher than the allowed number)";
-
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, msg);
- }
Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
.forCollection(collectionName)
.forShard(shardNames)
@@ -460,7 +411,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
.onNodes(nodeList)
.build();
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(cloudManager);
- Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, docCollection);
+ Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
replicaPositions = assignStrategy.assign(cloudManager, assignRequest);
}
@@ -482,7 +433,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
}
}
- public static DocCollection buildDocCollection(ZkNodeProps message, boolean withDocRouter) {
+ public static DocCollection buildDocCollection(DistribStateManager stateManager, ZkNodeProps message, boolean withDocRouter) {
log.info("buildDocCollection {}", message);
String cName = message.getStr(NAME);
DocRouter router = null;
@@ -549,6 +500,26 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
}
}
+ // TODO need to make this makePath calls efficient and not use zkSolrClient#makePath
+ for (String shardName : slices.keySet()) {
+ try {
+ stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/" + shardName, null, CreateMode.PERSISTENT, false);
+ // stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leader_elect", null, CreateMode.PERSISTENT, false);
+ stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leader_elect/" + shardName, null, CreateMode.PERSISTENT, false);
+ stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leader_elect/" + shardName + "/election", null, CreateMode.PERSISTENT, false);
+ stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leaders/" + shardName, null, CreateMode.PERSISTENT, false);
+ stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/terms/" + shardName, ZkStateReader.emptyJson, CreateMode.PERSISTENT, false);
+ } catch (AlreadyExistsException e) {
+ // okay
+ } catch (InterruptedException e) {
+ ParWork.propagateInterrupt(e);
+ throw new SolrException(ErrorCode.SERVER_ERROR, e);
+ } catch (KeeperException e) {
+ throw new SolrException(ErrorCode.SERVER_ERROR, e);
+ } catch (IOException e) {
+ throw new SolrException(ErrorCode.SERVER_ERROR, e);
+ }
+ }
DocCollection newCollection = new DocCollection(cName,
slices, collectionProps, router, 0);
@@ -710,7 +681,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
if (numShards == null)
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "numShards" + " is a required param");
for (int i = 0; i < numShards; i++) {
- final String sliceName = "shard" + (i + 1);
+ final String sliceName = "s" + (i + 1);
shardNames.add(sliceName);
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java
index e3aa49a..904a018 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java
@@ -18,9 +18,14 @@ package org.apache.solr.cloud.api.collections;
import java.lang.invoke.MethodHandles;
+import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
@@ -30,6 +35,9 @@ import org.apache.solr.common.params.CommonAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.admin.ConfigSetsHandlerApi;
+import org.apache.solr.handler.component.ShardHandler;
+import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -39,6 +47,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
+import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@@ -52,7 +61,11 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
- public Runnable call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
+ return addShard(clusterState, clusterState.getCollection(message.getStr(COLLECTION_PROP)), message, results);
+ }
+
+ AddReplicaCmd.Response addShard(ClusterState clusterState, DocCollection collection, ZkNodeProps message, NamedList results) throws Exception {
String extCollectionName = message.getStr(COLLECTION_PROP);
String sliceName = message.getStr(SHARD_ID_PROP);
boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
@@ -68,7 +81,6 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
} else {
collectionName = extCollectionName;
}
- DocCollection collection = clusterState.getCollection(collectionName);
int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
@@ -78,15 +90,10 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
}
- //ZkStateReader zkStateReader = ocmh.zkStateReader;
- ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
- // wait for a while until we see the shard
- //ocmh.waitForNewShard(collectionName, sliceName);
- // wait for a while until we see the shard and update the local view of the cluster state
- ocmh.waitForNewShard(collectionName, sliceName);
+ // create shard
+ clusterState = new CollectionMutator(ocmh.cloudManager).createShard(clusterState, message);
- // refresh clusterstate
- clusterState = ocmh.zkStateReader.getClusterState();
+ log.info("After create shard {}", clusterState);
String async = message.getStr(ASYNC);
ZkNodeProps addReplicasProps = new ZkNodeProps(
@@ -102,36 +109,87 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
ocmh.addPropertyParams(message, propertyParams);
addReplicasProps = addReplicasProps.plus(propertyParams);
if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
+
+ final String asyncId = message.getStr(ASYNC);
+ ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
+ OverseerCollectionMessageHandler.ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+
final NamedList addResult = new NamedList();
+ AddReplicaCmd.Response resp;
try {
//ocmh.addReplica(zkStateReader.getClusterState(), addReplicasProps, addResult, () -> {
- ocmh.addReplica(clusterState, addReplicasProps, addResult, () -> {
- Object addResultFailure = addResult.get("failure");
- if (addResultFailure != null) {
- SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
- if (failure == null) {
- failure = new SimpleOrderedMap();
- results.add("failure", failure);
- }
- failure.addAll((NamedList) addResultFailure);
- } else {
- SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
- if (success == null) {
- success = new SimpleOrderedMap();
- results.add("success", success);
- }
- success.addAll((NamedList) addResult.get("success"));
- }
- });
+ resp = new AddReplicaCmd(ocmh)
+ .addReplica(clusterState, addReplicasProps, shardHandler, shardRequestTracker, results); //ocmh.addReplica(clusterState, addReplicasProps, addResult).clusterState;
+ clusterState = resp.clusterState;
} catch (Assign.AssignmentException e) {
// clean up the slice that we created
- ZkNodeProps deleteShard = new ZkNodeProps(COLLECTION_PROP, collectionName, SHARD_ID_PROP, sliceName, ASYNC, async);
- new DeleteShardCmd(ocmh).call(clusterState, deleteShard, results);
+ // nocommit
+// ZkNodeProps deleteShard = new ZkNodeProps(COLLECTION_PROP, collectionName, SHARD_ID_PROP, sliceName, ASYNC, async);
+// new DeleteShardCmd(ocmh).call(clusterState, deleteShard, results);
throw e;
}
+// () -> {
+// Object addResultFailure = addResult.get("failure");
+// if (addResultFailure != null) {
+// SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
+// if (failure == null) {
+// failure = new SimpleOrderedMap();
+// results.add("failure", failure);
+// }
+// failure.addAll((NamedList) addResultFailure);
+// } else {
+// SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
+// if (success == null) {
+// success = new SimpleOrderedMap();
+// results.add("success", success);
+// }
+// success.addAll((NamedList) addResult.get("success"));
+// }
+// }
+
log.info("Finished create command on all shards for collection: {}", collectionName);
- return null;
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+ @Override
+ public AddReplicaCmd.Response call() {
+ try {
+ shardRequestTracker.processResponses(results, shardHandler, false, null, Collections.emptySet());
+ } catch (KeeperException e) {
+ log.error("", e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ } catch (InterruptedException e) {
+ ParWork.propagateInterrupt(e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+ // nocommit - put this in finalizer and finalizer after all calls to allow parallel and forward momentum
+
+ if (resp.asyncFinalRunner != null) {
+ try {
+ resp.asyncFinalRunner.call();
+ } catch (Exception e) {
+ log.error("Exception waiting for active replicas", e);
+ }
+ }
+
+ @SuppressWarnings({"rawtypes"}) boolean failure = results.get("failure") != null && ((SimpleOrderedMap) results.get("failure")).size() > 0;
+ if (failure) {
+
+ } else {
+
+ }
+
+ //ocmh.zkStateReader.waitForActiveCollection(collectionName, 10, TimeUnit.SECONDS, shardNames.size(), finalReplicaPositions.size());
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ return response;
+ }
+
+ };
+
+ response.clusterState = clusterState;
+ return response;
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java
index 79052c0..3e85afc 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java
@@ -66,7 +66,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked"})
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String extCollectionName = message.getStr(COLLECTION_PROP);
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
@@ -103,7 +103,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
for (Replica replica : slice.getReplicas()) {
if (replica.getState() != State.ACTIVE) {
if (log.isInfoEnabled()) {
- log.info("Replica {} is not active. Hence not sending the createsnapshot request", replica.getCoreName());
+ log.info("Replica {} is not active. Hence not sending the createsnapshot request", replica.getName());
}
continue; // Since replica is not active - no point sending a request.
}
@@ -142,7 +142,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
// to have latest state.
String coreName = (String)resp.get(CoreAdminParams.CORE);
Slice slice = shardByCoreName.remove(coreName);
- boolean leader = (slice.getLeader() != null && slice.getLeader().getCoreName().equals(coreName));
+ boolean leader = (slice.getLeader() != null && slice.getLeader().getName().equals(coreName));
resp.add(SolrSnapshotManager.SHARD_ID, slice.getName());
resp.add(SolrSnapshotManager.LEADER, leader);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java
index e65c071..59b08f8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java
@@ -33,7 +33,7 @@ public class DeleteAliasCmd implements OverseerCollectionMessageHandler.Cmd {
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String aliasName = message.getStr(NAME);
ZkStateReader zkStateReader = ocmh.zkStateReader;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java
index 8eca9ec..1666912 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java
@@ -19,12 +19,14 @@
package org.apache.solr.cloud.api.collections;
import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.common.NonExistentCoreException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.Aliases;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@@ -67,7 +69,7 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
log.info("delete collection called");
Object o = message.get(MaintainRoutedAliasCmd.INVOKED_BY_ROUTED_ALIAS);
if (o != null) {
@@ -104,14 +106,6 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
SolrZkClient zkClient = zkStateReader.getZkClient();
SolrSnapshotManager.cleanupCollectionLevelSnapshots(zkClient, collection);
- if (zkStateReader.getClusterState().getCollectionOrNull(collection) == null) {
- if (zkStateReader.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection)) {
- // if the collection is not in the clusterstate, but is listed in zk, do nothing, it will just
- // be removed in the finally - we cannot continue, because the below code will error if the collection
- // is not in the clusterstate
- return null;
- }
- }
// remove collection-level metrics history
if (deleteHistory) {
MetricsHistoryHandler historyHandler = ocmh.overseer.getCoreContainer().getMetricsHistoryHandler();
@@ -136,6 +130,8 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection");
}
+
+ clusterState = new ClusterStateMutator(ocmh.cloudManager) .deleteCollection(clusterState, collection);
@SuppressWarnings({"unchecked"})
List<Replica> failedReplicas = ocmh.collectionCmd(internalMsg, params, results, null, asyncId, okayExceptions);
@@ -144,18 +140,6 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
}
} finally {
- try {
- log.info("Send DELETE operation to Overseer collection={}", collection);
- ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETE.toLower(), NAME, collection);
- ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
-
- // wait for a while until we don't see the collection
- if (zkStateReader.getClusterState().getCollectionOrNull(collection) != null) {
- zkStateReader.waitForState(collection, 5, TimeUnit.SECONDS, (collectionState) -> collectionState == null);
- }
- } catch (Exception e) {
- log.error("Exception while removing collection", e);
- }
// make sure it's gone again after cores have been removed
try {
@@ -174,7 +158,22 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
});
}
}
- return null;
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ if (results.get("failure") == null && results.get("exception") == null) {
+
+ response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+ @Override
+ public AddReplicaCmd.Response call() {
+ // TODO: wait for delete collection?
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ return response;
+ }
+ };
+ }
+ response.clusterState = clusterState;
+ return response;
}
// This method returns the single collection aliases to delete, if present, or null
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java
index 73e6da8..5617b9a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java
@@ -51,7 +51,7 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked"})
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
ocmh.checkRequired(message, "node");
String node = message.getStr("node");
List<ZkNodeProps> sourceReplicas = ReplaceNodeCmd.getReplicasOfNode(node, state);
@@ -111,15 +111,8 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
NamedList deleteResult = new NamedList();
try {
if (async != null) sourceReplica = sourceReplica.plus(ASYNC, async);
- ((DeleteReplicaCmd) ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, sourceReplica.plus("parallel", "true"), deleteResult, () -> {
- if (deleteResult.get("failure") != null) {
- synchronized (results) {
-
- results.add("failure", String.format(Locale.ROOT, "Failed to delete replica for collection=%s shard=%s" +
- " on node=%s", coll, shard, node));
- }
- }
- });
+ // nocommit - return results from deleteReplica cmd
+ ((DeleteReplicaCmd) ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, sourceReplica.plus("parallel", "true"), deleteResult);
} catch (KeeperException e) {
log.warn("Error deleting ", e);
} catch (InterruptedException e) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java
index cd82c38..dc6c122 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java
@@ -21,14 +21,16 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
+import org.apache.solr.client.solrj.response.Cluster;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@@ -58,54 +60,63 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
public class DeleteReplicaCmd implements Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
+ private final boolean onlyUpdateState;
public DeleteReplicaCmd(OverseerCollectionMessageHandler ocmh) {
+ this.onlyUpdateState = false;
+ this.ocmh = ocmh;
+ }
+
+ public DeleteReplicaCmd(OverseerCollectionMessageHandler ocmh , boolean onlyUpdateState) {
+ this.onlyUpdateState = onlyUpdateState;
this.ocmh = ocmh;
}
@Override
@SuppressWarnings("unchecked")
- public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
- AddReplicaCmd.Response response = deleteReplica(clusterState, message, results, null);
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ AddReplicaCmd.Response response = deleteReplica(clusterState, message, results);
if (response == null) return null;
- return response.asyncFinalRunner;
+ return response;
}
@SuppressWarnings("unchecked")
- AddReplicaCmd.Response deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+ AddReplicaCmd.Response deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws KeeperException, InterruptedException {
log.info("deleteReplica() : {}", Utils.toJSONString(message));
//If a count is specified the strategy needs be different
if (message.getStr(COUNT_PROP) != null) {
- Set<String> pickReplicasTobeDeleted = deleteReplicaBasedOnCount(clusterState, message, results, onComplete);
- if (pickReplicasTobeDeleted != null && pickReplicasTobeDeleted.size() > 0) {
- if (results.get("failure") == null && results.get("exception") == null) {
- String collectionName = message.getStr(COLLECTION_PROP);
- String shard = message.getStr(SHARD_ID_PROP);
- AddReplicaCmd.Response response = new AddReplicaCmd.Response();
- pickReplicasTobeDeleted.forEach(replicaName -> {
-
- response.asyncFinalRunner = new Runnable() {
- @Override
- public void run() {
- try {
- waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
- } catch (Exception e) {
- log.error("", e);
- }
+ AddReplicaCmd.Response resp = deleteReplicaBasedOnCount(clusterState, message, results);
+ clusterState = resp.clusterState;
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ if (results.get("failure") == null && results.get("exception") == null) {
+ response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+ @Override
+ public AddReplicaCmd.Response call() {
+ if (resp.asyncFinalRunner != null) {
+ try {
+ resp.asyncFinalRunner.call();
+ } catch (Exception e) {
+ log.error("Exception running delete replica finalizers", e);
}
- };
- });
- return response;
- }
- return null;
- } else {
- return null;
+ }
+ // try {
+ // waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
+ // } catch (Exception e) {
+ // log.error("", e);
+ // }
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ return response;
+ }
+ };
}
+ response.clusterState = clusterState;
+ return response;
}
ocmh.checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP);
@@ -128,24 +139,38 @@ public class DeleteReplicaCmd implements Cmd {
"Invalid shard name : " + shard + " in collection : " + collectionName);
}
- deleteCore(slice, collectionName, replicaName, message, shard, results, onComplete);
- waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
+ AddReplicaCmd.Response resp = deleteCore(clusterState, slice, collectionName, replicaName, message, shard, results);
+ clusterState = resp.clusterState;
+
+ if (clusterState.getCollectionOrNull(collectionName).getReplica(replicaName) != null) {
+ throw new IllegalStateException("Failed to remove replica from state " + replicaName);
+ }
AddReplicaCmd.Response response = new AddReplicaCmd.Response();
if (results.get("failure") == null && results.get("exception") == null) {
- response.asyncFinalRunner = new Runnable() {
+ response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
@Override
- public void run() {
+ public AddReplicaCmd.Response call() {
+ if (resp.asyncFinalRunner != null) {
+ try {
+ resp.asyncFinalRunner.call();
+ } catch (Exception e) {
+ log.error("", e);
+ }
+ }
+
try {
waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
} catch (Exception e) {
log.error("", e);
}
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ return response;
}
};
}
-
+ response.clusterState = clusterState;
return response;
}
@@ -156,10 +181,9 @@ public class DeleteReplicaCmd implements Cmd {
* @return
*/
@SuppressWarnings({"unchecked"})
- Set<String> deleteReplicaBasedOnCount(ClusterState clusterState,
+ AddReplicaCmd.Response deleteReplicaBasedOnCount(ClusterState clusterState,
ZkNodeProps message,
- @SuppressWarnings({"rawtypes"})NamedList results,
- Runnable onComplete)
+ @SuppressWarnings({"rawtypes"})NamedList results)
throws KeeperException, InterruptedException {
ocmh.checkRequired(message, COLLECTION_PROP, COUNT_PROP);
int count = Integer.parseInt(message.getStr(COUNT_PROP));
@@ -171,15 +195,14 @@ public class DeleteReplicaCmd implements Cmd {
if (shard != null) {
slice = coll.getSlice(shard);
if (slice == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
- "Invalid shard name : " + shard + " in collection : " + collectionName);
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid shard name : " + shard + " in collection : " + collectionName);
}
}
Set<String> replicasToBeDeleted = null;
- Map<Slice, Set<String>> shardToReplicasMapping = new HashMap<Slice, Set<String>>();
+ Map<Slice,Set<String>> shardToReplicasMapping = new HashMap<Slice,Set<String>>();
if (slice != null) {
replicasToBeDeleted = pickReplicasTobeDeleted(slice, shard, collectionName, count);
- shardToReplicasMapping.put(slice,replicasToBeDeleted);
+ shardToReplicasMapping.put(slice, replicasToBeDeleted);
} else {
//If there are many replicas left, remove the rest based on count.
@@ -189,26 +212,40 @@ public class DeleteReplicaCmd implements Cmd {
shardToReplicasMapping.put(individualSlice, replicasToBeDeleted);
}
}
-
- try (ParWork worker = new ParWork(this, false, true)) {
-
- for (Map.Entry<Slice,Set<String>> entry : shardToReplicasMapping.entrySet()) {
- Slice shardSlice = entry.getKey();
- String shardId = shardSlice.getName();
- Set<String> replicas = entry.getValue();
- // callDeleteReplica on all replicas
- for (String replica : replicas) {
- if (log.isDebugEnabled()) log.debug("Deleting replica {} for shard {} based on count {}", replica, shardId, count);
- worker.collect("deleteCore", () -> {
- deleteCore(shardSlice, collectionName, replica, message, shard, results, onComplete); return replica;
-
- });
+ List<OverseerCollectionMessageHandler.Finalize> finalizers = new ArrayList<>();
+ for (Map.Entry<Slice,Set<String>> entry : shardToReplicasMapping.entrySet()) {
+ Slice shardSlice = entry.getKey();
+ String shardId = shardSlice.getName();
+ Set<String> replicas = entry.getValue();
+ // callDeleteReplica on all replicas
+ for (String replica : replicas) {
+ if (log.isDebugEnabled()) log.debug("Deleting replica {} for shard {} based on count {}", replica, shardId, count);
+ // nocommit - DONT DO THIS ONE AT TIME
+
+ AddReplicaCmd.Response resp = deleteCore(clusterState, shardSlice, collectionName, replica, message, shard, results);
+ clusterState = resp.clusterState;
+ if (resp.asyncFinalRunner != null) {
+ finalizers.add(resp.asyncFinalRunner);
}
- results.add("shard_id", shardId);
- results.add("replicas_deleted", replicas);
+
}
+ results.add("shard_id", shardId);
+ results.add("replicas_deleted", replicas);
}
- return replicasToBeDeleted;
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ response.clusterState = clusterState;
+ response.asyncFinalRunner = () -> {
+ AddReplicaCmd.Response resp = new AddReplicaCmd.Response();
+ resp.asyncFinalRunner = () -> {
+ for (OverseerCollectionMessageHandler.Finalize finalize : finalizers) {
+ finalize.call();
+ }
+ return new AddReplicaCmd.Response();
+ };
+ return resp;
+ };
+ return response;
}
@@ -225,12 +262,13 @@ public class DeleteReplicaCmd implements Cmd {
break;
}
//Try avoiding to pick up the leader to minimize activity on the cluster.
- if (leader.getCoreName().equals(replica.getCoreName())) {
+ if (leader.getName().equals(replica.getName())) {
continue;
}
replicasToBeRemoved.add(replica.getName());
count --;
}
+ log.info("Found replicas to be removed {}", replicasToBeRemoved);
return replicasToBeRemoved;
}
@@ -261,7 +299,7 @@ public class DeleteReplicaCmd implements Cmd {
}
@SuppressWarnings({"unchecked"})
- void deleteCore(Slice slice, String collectionName, String replicaName,ZkNodeProps message, String shard, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete) throws KeeperException, InterruptedException {
+ AddReplicaCmd.Response deleteCore(ClusterState clusterState, Slice slice, String collectionName, String replicaName, ZkNodeProps message, String shard, @SuppressWarnings({"rawtypes"})NamedList results) throws KeeperException, InterruptedException {
log.info("delete core {}", replicaName);
Replica replica = slice.getReplica(replicaName);
if (replica == null) {
@@ -274,61 +312,69 @@ public class DeleteReplicaCmd implements Cmd {
// If users are being safe and only want to remove a shard if it is down, they can specify onlyIfDown=true
// on the command.
- if (Boolean.parseBoolean(message.getStr(OverseerCollectionMessageHandler.ONLY_IF_DOWN)) && replica.getState() != Replica.State.DOWN) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
- "Attempted to remove replica : " + collectionName + "/" + shard + "/" + replicaName +
- " with onlyIfDown='true', but state is '" + replica.getStr(ZkStateReader.STATE_PROP) + "'");
- }
-
- ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
- String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
- String asyncId = message.getStr(ASYNC);
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.add(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.UNLOAD.toString());
- params.add(CoreAdminParams.CORE, core);
-
- params.set(CoreAdminParams.DELETE_INDEX, message.getBool(CoreAdminParams.DELETE_INDEX, true));
- params.set(CoreAdminParams.DELETE_INSTANCE_DIR, message.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, true));
- params.set(CoreAdminParams.DELETE_DATA_DIR, message.getBool(CoreAdminParams.DELETE_DATA_DIR, true));
- params.set(CoreAdminParams.DELETE_METRICS_HISTORY, message.getBool(CoreAdminParams.DELETE_METRICS_HISTORY, true));
-
- boolean isLive = ocmh.zkStateReader.getClusterState().getLiveNodes().contains(replica.getNodeName());
-
- try {
- ocmh.deleteCoreNode(collectionName, replicaName, replica, core);
- } catch (Exception e) {
- ParWork.propagateInterrupt(e);
- results.add("failure", "Could not complete delete " + e.getMessage());
- }
+// if (Boolean.parseBoolean(message.getStr(OverseerCollectionMessageHandler.ONLY_IF_DOWN)) && replica.getState() != Replica.State.DOWN) {
+// throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+// "Attempted to remove replica : " + collectionName + "/" + shard + "/" + replicaName +
+// " with onlyIfDown='true', but state is '" + replica.getStr(ZkStateReader.STATE_PROP) + "'");
+// }
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ ZkNodeProps rep = new ZkNodeProps();
+ rep.getProperties().put("replica", replicaName);
+ rep.getProperties().put("collection", replica.getCollection());
+ rep.getProperties().put(ZkStateReader.BASE_URL_PROP, replica.getBaseUrl());
+
+ log.info("Before slice remove replica {} {}", rep, clusterState);
+ clusterState = new SliceMutator(ocmh.cloudManager).removeReplica(clusterState, rep);
+ log.info("After slice remove replica {} {}", rep, clusterState);
+
+ if (!onlyUpdateState) {
+ ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+ String asyncId = message.getStr(ASYNC);
+
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.add(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.UNLOAD.toString());
+ params.add(CoreAdminParams.CORE, replicaName);
+
+ params.set(CoreAdminParams.DELETE_INDEX, message.getBool(CoreAdminParams.DELETE_INDEX, true));
+ params.set(CoreAdminParams.DELETE_INSTANCE_DIR, message.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, true));
+ params.set(CoreAdminParams.DELETE_DATA_DIR, message.getBool(CoreAdminParams.DELETE_DATA_DIR, true));
+ params.set(CoreAdminParams.DELETE_METRICS_HISTORY, message.getBool(CoreAdminParams.DELETE_METRICS_HISTORY, true));
+
+ boolean isLive = ocmh.zkStateReader.getClusterState().getLiveNodes().contains(replica.getNodeName());
+
+ // try {
+ // ocmh.deleteCoreNode(collectionName, replicaName, replica, core);
+ // } catch (Exception e) {
+ // ParWork.propagateInterrupt(e);
+ // results.add("failure", "Could not complete delete " + e.getMessage());
+ // }
+
+ final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+ if (isLive) {
+ response.asyncFinalRunner = () -> {
+ shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
+ return new AddReplicaCmd.Response();
+ };
- final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
- if (isLive) {
- shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
- }
+ }
- try {
try {
- if (isLive) {
- shardRequestTracker.processResponses(results, shardHandler, false, null);
- // try and ensure core info is removed from cluster state
- }
-
- } catch (Exception e) {
- ParWork.propagateInterrupt(e);
- results.add("failure", "Could not complete delete " + e.getMessage());
- } finally {
try {
- ocmh.deleteCoreNode(collectionName, replicaName, replica, core);
+ if (isLive) {
+ shardRequestTracker.processResponses(results, shardHandler, false, null);
+ // try and ensure core info is removed from cluster state
+ }
+
} catch (Exception e) {
ParWork.propagateInterrupt(e);
results.add("failure", "Could not complete delete " + e.getMessage());
}
- if (onComplete != null) onComplete.run();
+ } catch (Exception ex) {
+ throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Error waiting for corenode gone", ex);
}
- } catch (Exception ex) {
- throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Error waiting for corenode gone", ex);
}
+ response.clusterState = clusterState;
+ return response;
}
boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java
index aa9bc6c..40c6744 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java
@@ -23,13 +23,13 @@ import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
-import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.overseer.CollectionMutator;
import org.apache.solr.cloud.overseer.OverseerAction;
+import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@@ -67,7 +67,7 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked"})
- public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String extCollectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
String sliceId = message.getStr(ZkStateReader.SHARD_ID_PROP);
@@ -104,11 +104,13 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(sliceId, Slice.State.CONSTRUCTION.toString());
propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
ZkNodeProps m = new ZkNodeProps(propMap);
- ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
+ clusterState = new SliceMutator(ocmh.cloudManager).updateShardState(clusterState, message);
}
- String asyncId = message.getStr(ASYNC);
+ slice = clusterState.getCollection(collectionName).getSlice(sliceId);
+ String asyncId = message.getStr(ASYNC);
+ List<OverseerCollectionMessageHandler.Finalize> finalizers = new ArrayList<>();
try {
List<ZkNodeProps> replicas = getReplicasForSlice(collectionName, slice);
@@ -117,25 +119,15 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
if (log.isInfoEnabled()) {
log.info("Deleting replica for collection={} shard={} on node={}", replica.getStr(COLLECTION_PROP), replica.getStr(SHARD_ID_PROP), replica.getStr(CoreAdminParams.NODE));
}
- @SuppressWarnings({"rawtypes"})
- NamedList deleteResult = new NamedList();
+ @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
try {
- ((DeleteReplicaCmd)ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, replica, deleteResult, () -> {
-
- if (deleteResult.get("failure") != null) {
- synchronized (results) {
- results.add("failure", String.format(Locale.ROOT, "Failed to delete replica for collection=%s shard=%s" +
- " on node=%s", replica.getStr(COLLECTION_PROP), replica.getStr(SHARD_ID_PROP), replica.getStr(NODE_NAME_PROP)));
- }
- }
- @SuppressWarnings({"rawtypes"})
- SimpleOrderedMap success = (SimpleOrderedMap) deleteResult.get("success");
- if (success != null) {
- synchronized (results) {
- results.add("success", success);
- }
- }
- });
+
+ // nocommit - return results from deleteReplica cmd
+ AddReplicaCmd.Response resp = ((DeleteReplicaCmd) ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, replica, deleteResult);
+ if (resp.asyncFinalRunner != null) {
+ finalizers.add(resp.asyncFinalRunner);
+ }
+ clusterState = resp.clusterState;
} catch (KeeperException e) {
log.warn("Error deleting replica: {}", r, e);
throw e;
@@ -144,31 +136,11 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
log.warn("Error deleting replica: {}", r, e);
throw e;
}
- }
- log.debug("Waiting for delete shard action to complete");
-
- ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETESHARD.toLower(), ZkStateReader.COLLECTION_PROP,
- collectionName, ZkStateReader.SHARD_ID_PROP, sliceId);
- ZkStateReader zkStateReader = ocmh.zkStateReader;
- ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
-
-
- AddReplicaCmd.Response response = new AddReplicaCmd.Response();
-
- if (results.get("failure") == null && results.get("exception") == null) {
- response.asyncFinalRunner = new Runnable() {
- @Override
- public void run() {
- try {
- zkStateReader.waitForState(collectionName, 5, TimeUnit.SECONDS, (c) -> c != null && c.getSlice(sliceId) == null);
- } catch (InterruptedException e) {
- log.warn("", e);
- } catch (TimeoutException e) {
- log.warn("", e);
- }
- }
- };
- }
+ } log.debug("Waiting for delete shard action to complete");
+
+ ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETESHARD.toLower(), ZkStateReader.COLLECTION_PROP, collectionName, ZkStateReader.SHARD_ID_PROP, sliceId);
+
+ clusterState = new CollectionMutator(ocmh.cloudManager).deleteShard(clusterState, m);
log.info("Successfully deleted collection: {} , shard: {}", collectionName, sliceId);
} catch (SolrException e) {
@@ -178,7 +150,16 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Error executing delete operation for collection: " + collectionName + " shard: " + sliceId, e);
}
- return null;
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ response.asyncFinalRunner = () -> {
+ for (OverseerCollectionMessageHandler.Finalize finalize : finalizers) {
+ finalize.call();
+ }
+ return new AddReplicaCmd.Response();
+ };
+ response.clusterState = clusterState;
+ return response;
}
private List<ZkNodeProps> getReplicasForSlice(String collectionName, Slice slice) {
@@ -187,7 +168,6 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
ZkNodeProps props = new ZkNodeProps(
COLLECTION_PROP, collectionName,
SHARD_ID_PROP, slice.getName(),
- ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
ZkStateReader.REPLICA_PROP, replica.getName(),
CoreAdminParams.NODE, replica.getNodeName());
sourceReplicas.add(props);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java
index 08dddfc..7455ad3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java
@@ -64,7 +64,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked"})
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String extCollectionName = message.getStr(COLLECTION_PROP);
boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
String collectionName;
@@ -90,7 +90,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
Set<String> existingCores = new HashSet<>();
for (Slice s : ocmh.zkStateReader.getClusterState().getCollection(collectionName).getSlices()) {
for (Replica r : s.getReplicas()) {
- existingCores.add(r.getCoreName());
+ existingCores.add(r.getName());
}
}
@@ -112,7 +112,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
// Note - when a snapshot is found in_progress state - it is the result of overseer
// failure while handling the snapshot creation. Since we don't know the exact set of
// replicas to contact at this point, we try on all replicas.
- if (meta.get().getStatus() == SnapshotStatus.InProgress || coresWithSnapshot.contains(replica.getCoreName())) {
+ if (meta.get().getStatus() == SnapshotStatus.InProgress || coresWithSnapshot.contains(replica.getName())) {
String coreName = replica.getStr(CORE_NAME_PROP);
ModifiableSolrParams params = new ModifiableSolrParams();
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
index 3f32074..a48f2d9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
@@ -100,7 +100,7 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
}
@Override
- public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
//---- PARSE PRIMARY MESSAGE PARAMS
// important that we use NAME for the alias as that is what the Overseer will get a lock on before calling us
final String aliasName = message.getStr(NAME);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
index e5f87d9..8502762 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
@@ -45,7 +45,6 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandler;
-import org.apache.solr.handler.component.ShardHandlerFactory;
import org.apache.solr.update.SolrIndexSplitter;
import org.apache.solr.util.TimeOut;
import org.slf4j.Logger;
@@ -74,7 +73,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
- public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String extSourceCollectionName = message.getStr("collection");
String splitKey = message.getStr("split.key");
String extTargetCollectionName = message.getStr("target.collection");
@@ -252,9 +251,8 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
Slice tempSourceSlice = clusterState.getCollection(tempSourceCollectionName).getSlices().iterator().next();
Replica tempSourceLeader = zkStateReader.getLeaderRetry(tempSourceCollectionName, tempSourceSlice.getName(), 120000);
- String tempCollectionReplica1 = tempSourceLeader.getCoreName();
- String coreNodeName = ocmh.waitForCoreNodeName(zkStateReader, tempSourceCollectionName,
- sourceLeader.getNodeName(), tempCollectionReplica1);
+ String tempCollectionReplica1 = tempSourceLeader.getName();
+
// wait for the replicas to be seen as active on temp source leader
if (log.isInfoEnabled()) {
log.info("Asking source leader to wait for: {} to be alive on: {}", tempCollectionReplica1, sourceLeader.getNodeName());
@@ -262,10 +260,11 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
cmd.setCoreName(tempCollectionReplica1);
cmd.setNodeName(sourceLeader.getNodeName());
- cmd.setCoreNodeName(coreNodeName);
cmd.setState(Replica.State.ACTIVE);
cmd.setCheckLive(true);
cmd.setOnlyIfLeader(true);
+ cmd.setCollection(tempSourceCollectionName);
+ cmd.setShardId(tempSourceSlice.getName());
{
final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
// we don't want this to happen asynchronously
@@ -280,8 +279,8 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
log.info("Asking source leader to split index");
params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.SPLIT.toString());
- params.set(CoreAdminParams.CORE, sourceLeader.getStr("core"));
- params.add(CoreAdminParams.TARGET_CORE, tempSourceLeader.getStr("core"));
+ params.set(CoreAdminParams.CORE, sourceLeader.getName());
+ params.add(CoreAdminParams.TARGET_CORE, tempSourceLeader.getName());
params.set(CoreAdminParams.RANGES, splitRange.toString());
params.set("split.key", splitKey);
@@ -296,8 +295,14 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
tempSourceCollectionName, targetLeader.getNodeName());
}
- String tempCollectionReplica2 = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
- zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
+ int replicas = 0;
+ DocCollection docCollection = clusterState.getCollectionOrNull(tempSourceCollectionName);
+ if (docCollection != null) {
+ replicas = docCollection.getReplicas().size();
+ }
+
+ String tempCollectionReplica2 = Assign.buildSolrCoreName(docCollection,
+ tempSourceCollectionName, tempSourceSlice.getName(), Replica.Type.NRT);
props = new HashMap<>();
props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
props.put(COLLECTION_PROP, tempSourceCollectionName);
@@ -314,15 +319,14 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
if (asyncId != null) {
props.put(ASYNC, asyncId);
}
- ((AddReplicaCmd)ocmh.commandMap.get(ADDREPLICA)).addReplica(clusterState, new ZkNodeProps(props), results, null);
+ ((AddReplicaCmd)ocmh.commandMap.get(ADDREPLICA)).call(clusterState, new ZkNodeProps(props), results);
{
final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
syncRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to create replica of " +
"temporary collection in target leader node.");
}
- coreNodeName = ocmh.waitForCoreNodeName(zkStateReader, tempSourceCollectionName,
- targetLeader.getNodeName(), tempCollectionReplica2);
+
// wait for the replicas to be seen as active on temp source leader
if (log.isInfoEnabled()) {
log.info("Asking temp source leader to wait for: {} to be alive on: {}", tempCollectionReplica2, targetLeader.getNodeName());
@@ -330,7 +334,6 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
cmd = new CoreAdminRequest.WaitForState();
cmd.setCoreName(tempSourceLeader.getStr("core"));
cmd.setNodeName(targetLeader.getNodeName());
- cmd.setCoreNodeName(coreNodeName);
cmd.setState(Replica.State.ACTIVE);
cmd.setCheckLive(true);
cmd.setOnlyIfLeader(true);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyCollectionCmd.java
new file mode 100644
index 0000000..3527da7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyCollectionCmd.java
@@ -0,0 +1,28 @@
+package org.apache.solr.cloud.api.collections;
+
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+
+public class ModifyCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
+
+ private final OverseerCollectionMessageHandler ocmh;
+ private final TimeSource timeSource;
+
+ public ModifyCollectionCmd(OverseerCollectionMessageHandler ocmh) {
+ this.ocmh = ocmh;
+ this.timeSource = ocmh.cloudManager.getTimeSource();
+ }
+
+ @Override
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
+
+ clusterState = new CollectionMutator(ocmh.cloudManager).modifyCollection(clusterState, message);
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ response.clusterState = clusterState;
+ return response;
+ }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
index 8182c41..20bcfd9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
@@ -23,8 +23,8 @@ import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
-import org.apache.solr.cloud.ActiveReplicaWatcher;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrCloseableLatch;
import org.apache.solr.common.SolrException;
@@ -67,12 +67,11 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
- moveReplica(ocmh.zkStateReader.getClusterState(), message, results);
- return null;
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
+ return moveReplica(ocmh.zkStateReader.getClusterState(), message, results);
}
- private void moveReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ private AddReplicaCmd.Response moveReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
if (log.isDebugEnabled()) {
log.debug("moveReplica() : {}", Utils.toJSONString(message));
}
@@ -105,14 +104,12 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
String replicaName = message.getStr(REPLICA_PROP);
replica = coll.getReplica(replicaName);
if (replica == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
- "Collection: " + collection + " replica: " + replicaName + " does not exist");
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " replica: " + replicaName + " does not exist");
}
} else {
String sourceNode = message.getStr(CollectionParams.SOURCE_NODE, message.getStr(CollectionParams.FROM_NODE));
if (sourceNode == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'" + CollectionParams.SOURCE_NODE +
- " or '" + CollectionParams.FROM_NODE + "' is a required param");
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'" + CollectionParams.SOURCE_NODE + " or '" + CollectionParams.FROM_NODE + "' is a required param");
}
String shardId = message.getStr(SHARD_ID_PROP);
if (shardId == null) {
@@ -121,8 +118,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
Slice slice = coll.getSlice(shardId);
List<Replica> sliceReplicas = new ArrayList<>(slice.getReplicas(r -> sourceNode.equals(r.getNodeName())));
if (sliceReplicas.isEmpty()) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
- "Collection: " + collection + " node: " + sourceNode + " does not have any replica belonging to shard: " + shardId);
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " node: " + sourceNode + " does not have any replica belonging to shard: " + shardId);
}
Collections.shuffle(sliceReplicas, OverseerCollectionMessageHandler.RANDOM);
replica = sliceReplicas.iterator().next();
@@ -140,8 +136,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
List<Replica> replicasOnSourceNode = coll.getReplicas(replica.getNodeName());
if (replicasOnSourceNode == null || replicasOnSourceNode.size() < 2) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
- "Collection: " + collection + " is co-located with collection: " + colocatedCollectionName
- + " and has a single replica: " + replica.getName() + " on node: " + replica.getNodeName()
+ "Collection: " + collection + " is co-located with collection: " + colocatedCollectionName + " and has a single replica: " + replica.getName() + " on node: " + replica.getNodeName()
+ " so it is not possible to move it to another node");
}
}
@@ -158,41 +153,58 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
assert slice != null;
Object dataDir = replica.get("dataDir");
boolean isSharedFS = replica.getBool(ZkStateReader.SHARED_STORAGE_PROP, false) && dataDir != null;
-
+ OverseerCollectionMessageHandler.Finalize finalizer = null;
if (isSharedFS && inPlaceMove) {
log.debug("-- moveHdfsReplica");
moveHdfsReplica(clusterState, results, dataDir.toString(), targetNode, async, coll, replica, slice, timeout, waitForFinalState);
} else {
log.debug("-- moveNormalReplica (inPlaceMove={}, isSharedFS={}", inPlaceMove, isSharedFS);
- moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice, timeout, waitForFinalState);
+ finalizer = moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice, timeout, waitForFinalState);
}
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ OverseerCollectionMessageHandler.Finalize finalIzer = finalizer;
+ response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+ @Override
+ public AddReplicaCmd.Response call() {
+ if (finalIzer != null) {
+ try {
+ finalIzer.call();
+ } catch (Exception e) {
+ log.error("Exception during MoveReplica", e);
+ }
+ }
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ return response;
+ }
+ };
+
+ response.clusterState = clusterState;
+
+ return response;
}
@SuppressWarnings({"unchecked"})
- private void moveHdfsReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"})NamedList results, String dataDir, String targetNode, String async,
- DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
+ private void moveHdfsReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"}) NamedList results, String dataDir, String targetNode, String async, DocCollection coll, Replica replica,
+ Slice slice, int timeout, boolean waitForFinalState) throws Exception {
String skipCreateReplicaInClusterState = "true";
if (clusterState.getLiveNodes().contains(replica.getNodeName())) {
skipCreateReplicaInClusterState = "false";
- ZkNodeProps removeReplicasProps = new ZkNodeProps(
- COLLECTION_PROP, coll.getName(),
- SHARD_ID_PROP, slice.getName(),
- REPLICA_PROP, replica.getName()
- );
+ ZkNodeProps removeReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), REPLICA_PROP, replica.getName());
removeReplicasProps.getProperties().put(CoreAdminParams.DELETE_DATA_DIR, false);
removeReplicasProps.getProperties().put(CoreAdminParams.DELETE_INDEX, false);
if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
- @SuppressWarnings({"rawtypes"})
- NamedList deleteResult = new NamedList();
+ @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
try {
- ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
+ ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult);
} catch (SolrException e) {
// assume this failed completely so there's nothing to roll back
deleteResult.add("failure", e.toString());
}
if (deleteResult.get("failure") != null) {
- String errorString = String.format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s",
- coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
+ String errorString = String
+ .format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s", coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
log.warn(errorString);
results.add("failure", errorString);
return;
@@ -215,124 +227,118 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
String ulogDir = replica.getStr(CoreAdminParams.ULOG_DIR);
- ZkNodeProps addReplicasProps = new ZkNodeProps(
- COLLECTION_PROP, coll.getName(),
- SHARD_ID_PROP, slice.getName(),
- CoreAdminParams.NODE, targetNode,
- CoreAdminParams.CORE_NODE_NAME, replica.getName(),
- CoreAdminParams.NAME, replica.getCoreName(),
- WAIT_FOR_FINAL_STATE, String.valueOf(waitForFinalState),
- SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, skipCreateReplicaInClusterState,
- CoreAdminParams.ULOG_DIR, ulogDir.substring(0, ulogDir.lastIndexOf(UpdateLog.TLOG_NAME)),
- CoreAdminParams.DATA_DIR, dataDir,
- ZkStateReader.REPLICA_TYPE, replica.getType().name());
+ ZkNodeProps addReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), CoreAdminParams.NODE, targetNode, CoreAdminParams.NAME, replica.getName(),
+ WAIT_FOR_FINAL_STATE, String.valueOf(waitForFinalState), SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, skipCreateReplicaInClusterState, CoreAdminParams.ULOG_DIR,
+ ulogDir.substring(0, ulogDir.lastIndexOf(UpdateLog.TLOG_NAME)), CoreAdminParams.DATA_DIR, dataDir, ZkStateReader.REPLICA_TYPE, replica.getType().name());
- if(async!=null) addReplicasProps.getProperties().put(ASYNC, async);
- @SuppressWarnings({"rawtypes"})
- NamedList addResult = new NamedList();
+ if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
+ @SuppressWarnings({"rawtypes"}) NamedList addResult = new NamedList();
try {
- ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, addResult, null);
+ ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, addResult);
} catch (Exception e) {
ParWork.propagateInterrupt(e);
// fatal error - try rolling back
- String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
- " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
+ String errorString = String
+ .format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" + " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
results.add("failure", errorString);
- log.warn("Error adding replica {} - trying to roll back...", addReplicasProps, e);
+ log.warn("Error adding replica {} - trying to roll back...", addReplicasProps, e);
addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNodeName());
- @SuppressWarnings({"rawtypes"})
- NamedList rollback = new NamedList();
- ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback, null);
+ @SuppressWarnings({"rawtypes"}) NamedList rollback = new NamedList();
+ ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback);
if (rollback.get("failure") != null) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
- + ", collection may be inconsistent: " + rollback.get("failure"));
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica + ", collection may be inconsistent: " + rollback.get("failure"));
}
return;
}
if (addResult.get("failure") != null) {
- String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
- " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
+ String errorString = String
+ .format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" + " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
log.warn(errorString);
results.add("failure", errorString);
log.debug("--- trying to roll back...");
// try to roll back
addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNodeName());
- @SuppressWarnings({"rawtypes"})
- NamedList rollback = new NamedList();
+ @SuppressWarnings({"rawtypes"}) NamedList rollback = new NamedList();
try {
- ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback, null);
+ ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback);
} catch (Exception e) {
ParWork.propagateInterrupt(e);
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
- + ", collection may be inconsistent!", e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica + ", collection may be inconsistent!", e);
}
if (rollback.get("failure") != null) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
- + ", collection may be inconsistent! Failure: " + rollback.get("failure"));
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica + ", collection may be inconsistent! Failure: " + rollback.get("failure"));
}
return;
} else {
- String successString = String.format(Locale.ROOT, "MOVEREPLICA action completed successfully, moved replica=%s at node=%s " +
- "to replica=%s at node=%s", replica.getCoreName(), replica.getNodeName(), replica.getCoreName(), targetNode);
+ String successString = String
+ .format(Locale.ROOT, "MOVEREPLICA action completed successfully, moved replica=%s at node=%s " + "to replica=%s at node=%s", replica.getName(), replica.getNodeName(), replica.getName(),
+ targetNode);
results.add("success", successString);
}
}
@SuppressWarnings({"unchecked"})
- private void moveNormalReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"})NamedList results, String targetNode, String async,
- DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
- String newCoreName = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
- ZkNodeProps addReplicasProps = new ZkNodeProps(
- COLLECTION_PROP, coll.getName(),
- SHARD_ID_PROP, slice.getName(),
- CoreAdminParams.NODE, targetNode,
- CoreAdminParams.NAME, newCoreName,
+ private OverseerCollectionMessageHandler.Finalize moveNormalReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"}) NamedList results, String targetNode, String async, DocCollection coll,
+ Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
+ String newCoreName = Assign.buildSolrCoreName(coll, coll.getName(), slice.getName(), replica.getType());
+ ZkNodeProps addReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), CoreAdminParams.NODE, targetNode, CoreAdminParams.NAME, newCoreName,
ZkStateReader.REPLICA_TYPE, replica.getType().name());
if (async != null) addReplicasProps.getProperties().put(ASYNC, async + "-AddReplica-" + Math.abs(System.nanoTime()));
- @SuppressWarnings({"rawtypes"})
- NamedList addResult = new NamedList();
+ @SuppressWarnings({"rawtypes"}) NamedList addResult = new NamedList();
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(1, ocmh);
- AddReplicaCmd.Response response = ocmh.addReplicaWithResp(clusterState, addReplicasProps, addResult, null);
+ AddReplicaCmd.Response response = ocmh.addReplicaWithResp(clusterState, addReplicasProps, addResult);
+
+ ocmh.overseer.getZkStateWriter().enqueueUpdate(response.clusterState, false);
+ ocmh.overseer.writePendingUpdates();
+
- if (addResult.get("failure") != null) {
- String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
- " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
- log.warn(errorString);
- results.add("failure", errorString);
- return;
- }
// wait for the other replica to be active if the source replica was a leader
- log.debug("Waiting for leader's replica to recover.");
- if (replica.equals(slice.getLeader()) || waitForFinalState) {
- if (response != null && response.asyncFinalRunner != null) {
- response.asyncFinalRunner.run();
- }
- }
- ZkNodeProps removeReplicasProps = new ZkNodeProps(
- COLLECTION_PROP, coll.getName(),
- SHARD_ID_PROP, slice.getName(),
- REPLICA_PROP, replica.getName());
- if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
- @SuppressWarnings({"rawtypes"})
- NamedList deleteResult = new NamedList();
- try {
- ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
- } catch (SolrException e) {
- deleteResult.add("failure", e.toString());
- }
- if (deleteResult.get("failure") != null) {
- String errorString = String.format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s",
- coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
- log.warn(errorString);
- results.add("failure", errorString);
- } else {
- String successString = String.format(Locale.ROOT, "MOVEREPLICA action completed successfully, moved replica=%s at node=%s " +
- "to replica=%s at node=%s", replica.getCoreName(), replica.getNodeName(), newCoreName, targetNode);
- results.add("success", successString);
- }
+
+ return new OverseerCollectionMessageHandler.Finalize() {
+ @Override
+ public AddReplicaCmd.Response call() throws Exception {
+ log.debug("Waiting for leader's replica to recover.");
+
+ response.asyncFinalRunner.call();
+
+ if (addResult.get("failure") != null) {
+ String errorString = String
+ .format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" + " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
+ log.warn(errorString);
+ results.add("failure", errorString);
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ return response;
+ } else {
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ ZkNodeProps removeReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), REPLICA_PROP, replica.getName());
+ if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
+ @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
+ try {
+ response.clusterState = ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult).clusterState;
+ } catch (SolrException e) {
+ deleteResult.add("failure", e.toString());
+ }
+ if (deleteResult.get("failure") != null) {
+ String errorString = String.format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s", coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
+ log.warn(errorString);
+ results.add("failure", errorString);
+ } else {
+ String successString = String
+ .format(Locale.ROOT, "MOVEREPLICA action completed successfully, moved replica=%s at node=%s " + "to replica=%s at node=%s", replica.getName(), replica.getNodeName(), newCoreName,
+ targetNode);
+ results.add("success", successString);
+ }
+
+ return response;
+ }
+ }
+ };
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
index 8c10349..d7222ea 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
@@ -31,6 +31,7 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
@@ -48,21 +49,18 @@ import org.apache.solr.client.solrj.impl.Http2SolrClient;
import org.apache.solr.client.solrj.impl.LBHttp2SolrClient;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.cloud.LockTree;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.OverseerMessageHandler;
-import org.apache.solr.cloud.OverseerNodePrioritizer;
import org.apache.solr.cloud.OverseerSolrResponse;
-import org.apache.solr.cloud.OverseerSolrResponseSerializer;
import org.apache.solr.cloud.OverseerTaskProcessor;
import org.apache.solr.cloud.Stats;
import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.overseer.CollectionMutator;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.ParWork;
-import org.apache.solr.common.PerThreadExecService;
import org.apache.solr.common.SolrCloseable;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@@ -72,7 +70,6 @@ import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkConfigManager;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
@@ -80,13 +77,15 @@ import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
-import org.apache.solr.handler.component.HttpShardHandler;
+import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardRequest;
@@ -95,10 +94,8 @@ import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
-import org.eclipse.jetty.util.BlockingArrayQueue;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -110,7 +107,6 @@ import static org.apache.solr.client.solrj.response.RequestStatusState.SUBMITTED
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
@@ -178,7 +174,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
// This is used for handling mutual exclusion of the tasks.
final private LockTree lockTree = new LockTree();
- ExecutorService tpe = ParWork.getParExecutorService("overseerTPE", 0, 16, 0, new BlockingArrayQueue());
+
+ ExecutorService tpe = new ExecutorUtil.MDCAwareThreadPoolExecutor(5, 10, 0L, TimeUnit.MILLISECONDS,
+ new SynchronousQueue<>(),
+ new SolrNamedThreadFactory("OverseerCollectionMessageHandlerThreadFactory"));
public static final Random RANDOM;
static {
@@ -196,17 +195,15 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
private volatile boolean isClosed;
- public OverseerCollectionMessageHandler(ZkStateReader zkStateReader, String myId,
+ public OverseerCollectionMessageHandler(CoreContainer cc, String myId,
LBHttp2SolrClient overseerLbClient,
- HttpShardHandlerFactory shardHandlerFactory,
String adminPath,
Stats stats,
- Overseer overseer,
- OverseerNodePrioritizer overseerPrioritizer) {
+ Overseer overseer) {
// TODO: can leak single instance of this oddly in AddReplicaTest
// assert ObjectReleaseTracker.track(this);
- this.zkStateReader = zkStateReader;
- this.shardHandlerFactory = shardHandlerFactory;
+ this.zkStateReader = cc.getZkController().getZkStateReader();
+ this.shardHandlerFactory = (HttpShardHandlerFactory) cc.getShardHandlerFactory();
this.overseerLbClient = overseerLbClient;
this.adminPath = adminPath;
this.myId = myId;
@@ -223,8 +220,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
.put(CREATESNAPSHOT, new CreateSnapshotCmd(this))
.put(DELETESNAPSHOT, new DeleteSnapshotCmd(this))
.put(SPLITSHARD, new SplitShardCmd(this))
- .put(ADDROLE, new OverseerRoleCmd(this, ADDROLE, overseerPrioritizer))
- .put(REMOVEROLE, new OverseerRoleCmd(this, REMOVEROLE, overseerPrioritizer))
.put(MOCK_COLL_TASK, this::mockOperation)
.put(MOCK_SHARD_TASK, this::mockOperation)
.put(MOCK_REPLICA_TASK, this::mockOperation)
@@ -260,7 +255,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
MDCLoggingContext.setShard(message.getStr(SHARD_ID_PROP));
MDCLoggingContext.setReplica(message.getStr(REPLICA_PROP));
log.debug("OverseerCollectionMessageHandler.processMessage : {} , {}", operation, message);
-
+ ClusterState clusterState = overseer.getZkStateWriter().getClusterstate(false);
@SuppressWarnings({"rawtypes"})
NamedList results = new NamedList();
@@ -268,10 +263,41 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
CollectionAction action = getCollectionAction(operation);
Cmd command = commandMap.get(action);
if (command != null) {
- Runnable runFinal = command.call(overseer.getZkStateReader().getClusterState(), message, results);
- if (runFinal != null && results.get("failure") == null && results.get("exception") == null) {
- runFinal.run();
+ AddReplicaCmd.Response responce = command.call(clusterState, message, results);
+ if (responce == null || responce.clusterState == null) {
+ throw new SolrException(ErrorCode.SERVER_ERROR, "CMD did not return a new clusterstate:" + operation);
+ }
+
+ log.info("Command returned clusterstate={} results={}", responce.clusterState, results);
+
+ overseer.getZkStateWriter().enqueueUpdate(responce.clusterState, false);
+
+ overseer.writePendingUpdates();
+
+
+ // nocommit consider
+ if (responce != null && responce.asyncFinalRunner != null && results.get("failure") == null && results.get("exception") == null) {
+ AddReplicaCmd.Response resp = responce.asyncFinalRunner.call();
+ log.info("Finalize after Command returned clusterstate={}", resp.clusterState);
+ if (resp.clusterState != null) {
+ overseer.getZkStateWriter().enqueueUpdate(responce.clusterState, false);
+ overseer.writePendingUpdates();
+ }
}
+
+ String collection = message.getStr("collection");
+ if (collection == null) {
+ collection = message.getStr("name");
+ }
+ if (collection != null) {
+ Integer version = overseer.getZkStateWriter().lastWrittenVersion(collection);
+ if (version != null && !action.equals(DELETE)) {
+ results.add("csver", version); // nocommit - find out which version was written by overseer and return it in response for this
+ } else {
+ //deleted
+ }
+ }
+
} else {
throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
+ operation);
@@ -305,13 +331,13 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
nl.add("rspCode", e instanceof SolrException ? ((SolrException)e).code() : -1);
results.add("exception", nl);
}
- // results.addAll(threadSafeResults);
+
return new OverseerSolrResponse(results);
}
@SuppressForbidden(reason = "Needs currentTimeMillis for mock requests")
@SuppressWarnings({"unchecked"})
- private Runnable mockOperation(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws InterruptedException {
+ private AddReplicaCmd.Response mockOperation(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws InterruptedException {
//only for test purposes
Thread.sleep(message.getInt("sleep", 1));
if (log.isInfoEnabled()) {
@@ -330,20 +356,25 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
@SuppressWarnings({"unchecked"})
- private Runnable reloadCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws KeeperException, InterruptedException {
+ private AddReplicaCmd.Response reloadCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws KeeperException, InterruptedException {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminAction.RELOAD.toString());
String asyncId = message.getStr(ASYNC);
collectionCmd(message, params, results, Replica.State.ACTIVE, asyncId);
- return null;
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ response.results = results;
+ // nocommit - we don't change this for this cmd, we should be able to indicate that to caller
+ response.clusterState = clusterState;
+ return response;
}
@SuppressWarnings("unchecked")
- private Runnable processRebalanceLeaders(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+ private AddReplicaCmd.Response processRebalanceLeaders(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, CORE_NAME_PROP, ELECTION_NODE_PROP,
- CORE_NODE_NAME_PROP, BASE_URL_PROP, REJOIN_AT_HEAD_PROP);
+ BASE_URL_PROP, REJOIN_AT_HEAD_PROP);
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(COLLECTION_PROP, message.getStr(COLLECTION_PROP));
@@ -351,7 +382,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
params.set(REJOIN_AT_HEAD_PROP, message.getStr(REJOIN_AT_HEAD_PROP));
params.set(CoreAdminParams.ACTION, CoreAdminAction.REJOINLEADERELECTION.toString());
params.set(CORE_NAME_PROP, message.getStr(CORE_NAME_PROP));
- params.set(CORE_NODE_NAME_PROP, message.getStr(CORE_NODE_NAME_PROP));
params.set(ELECTION_NODE_PROP, message.getStr(ELECTION_NODE_PROP));
params.set(BASE_URL_PROP, message.getStr(BASE_URL_PROP));
@@ -370,7 +400,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
@SuppressWarnings("unchecked")
- private Runnable processReplicaAddPropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+ private AddReplicaCmd.Response processReplicaAddPropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, ZkStateReader.NUM_SHARDS_PROP, "shards", REPLICA_PROP, PROPERTY_PROP, PROPERTY_VALUE_PROP);
Map<String, Object> propMap = new HashMap<>(message.getProperties().size() + 1);
@@ -381,7 +411,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
return null;
}
- private Runnable processReplicaDeletePropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+ private AddReplicaCmd.Response processReplicaDeletePropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP);
Map<String, Object> propMap = new HashMap<>(message.getProperties().size() + 1);
@@ -392,7 +422,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
return null;
}
- private Runnable balanceProperty(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ private AddReplicaCmd.Response balanceProperty(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
if (StringUtils.isBlank(message.getStr(COLLECTION_PROP)) || StringUtils.isBlank(message.getStr(PROPERTY_PROP))) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"The '" + COLLECTION_PROP + "' and '" + PROPERTY_PROP +
@@ -438,10 +468,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
@SuppressWarnings("unchecked")
- void deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+ AddReplicaCmd.Response deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
- ((DeleteReplicaCmd) commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, message, results, onComplete);
-
+ return ((DeleteReplicaCmd) commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, message, results);
}
void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws Exception {
@@ -450,7 +479,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
ZkStateReader.CORE_NAME_PROP, core,
ZkStateReader.NODE_NAME_PROP, replica.getStr(ZkStateReader.NODE_NAME_PROP),
ZkStateReader.COLLECTION_PROP, collectionName,
- ZkStateReader.CORE_NODE_NAME_PROP, replicaName,
ZkStateReader.BASE_URL_PROP, replica.getStr(ZkStateReader.BASE_URL_PROP));
overseer.offerStateUpdate(Utils.toJSON(m));
}
@@ -482,7 +510,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
@SuppressWarnings({"unchecked"})
void commit(@SuppressWarnings({"rawtypes"})NamedList results, String slice, Replica parentShardLeader) {
log.debug("Calling soft commit to make sub shard updates visible");
- String coreUrl = new ZkCoreNodeProps(parentShardLeader).getCoreUrl();
+ String coreUrl = parentShardLeader.getCoreUrl();
// HttpShardHandler is hard coded to send a QueryRequest hence we go direct
// and we force open a searcher so that we have documents to show upon switching states
UpdateResponse updateResponse = null;
@@ -505,41 +533,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
return ureq.process(httpClient);
}
- static String waitForCoreNodeName(ZkStateReader zkStateReader, String collectionName, String msgNodeName, String msgCore) {
- AtomicReference<String> errorMessage = new AtomicReference<>();
- AtomicReference<String> coreNodeName = new AtomicReference<>();
- try {
- zkStateReader.waitForState(collectionName, 15, TimeUnit.SECONDS, (n, c) -> {
- if (c == null)
- return false;
- final Map<String,Slice> slicesMap = c.getSlicesMap();
- for (Slice slice : slicesMap.values()) {
- for (Replica replica : slice.getReplicas()) {
-
- String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
- String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
-
- if (msgNodeName.equals(nodeName) && core.equals(msgCore)) {
- coreNodeName.set(replica.getName());
- return true;
- }
- }
- }
- return false;
- });
- } catch (TimeoutException e) {
- String error = errorMessage.get();
- if (error == null)
- error = "Timeout waiting for collection state.";
- throw new ZkController.NotInClusterStateException(ErrorCode.SERVER_ERROR, error);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted", e);
- }
-
- return coreNodeName.get();
- }
-
void waitForNewShard(String collectionName, String sliceName) {
log.debug("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
try {
@@ -594,7 +587,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
- private Runnable modifyCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+ private AddReplicaCmd.Response modifyCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
@@ -602,55 +595,31 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
//remove the property here to avoid any errors down the pipeline due to this property appearing
String configName = (String) message.getProperties().remove(CollectionAdminParams.COLL_CONF);
- if(configName != null) {
+ if (configName != null) {
validateConfigOrThrowSolrException(configName);
createConfNode(cloudManager.getDistribStateManager(), configName, collectionName);
reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
}
- overseer.offerStateUpdate(Utils.toJSON(message));
-
- try {
- zkStateReader.waitForState(collectionName, 30, TimeUnit.SECONDS, (n, c) -> {
- if (c == null) return false;
-
- for (Map.Entry<String,Object> updateEntry : message.getProperties().entrySet()) {
- String updateKey = updateEntry.getKey();
-
- if (!updateKey.equals(ZkStateReader.COLLECTION_PROP)
- && !updateKey.equals(Overseer.QUEUE_OPERATION)
- && updateEntry.getValue() != null // handled below in a separate conditional
- && !updateEntry.getValue().equals(c.get(updateKey))) {
- return false;
- }
-
- if (updateEntry.getValue() == null && c.containsKey(updateKey)) {
- return false;
- }
- }
- return true;
- });
- } catch (TimeoutException | InterruptedException e) {
- ParWork.propagateInterrupt(e);
- log.error("modifyCollection(ClusterState=" + clusterState + ", ZkNodeProps=" + message + ", NamedList=" + results + ")", e);
- throw new SolrException(ErrorCode.SERVER_ERROR, "Could not modify collection " + message, e);
- }
+ clusterState = new CollectionMutator(cloudManager).modifyCollection(clusterState, message);
// if switching to/from read-only mode reload the collection
if (message.keySet().contains(ZkStateReader.READ_ONLY)) {
reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
}
- return null;
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ response.clusterState = clusterState;
+ return response;
}
- void cleanupCollection(String collectionName, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ AddReplicaCmd.Response cleanupCollection(String collectionName, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
log.error("Cleaning up collection [{}].", collectionName);
Map<String, Object> props = makeMap(
Overseer.QUEUE_OPERATION, DELETE.toLower(),
NAME, collectionName);
- commandMap.get(DELETE).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
- zkStateReader.waitForState(collectionName, 10, TimeUnit.SECONDS, (liveNodes, collectionState) -> collectionState == null);
+ AddReplicaCmd.Response response = commandMap.get(DELETE).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
+ return response;
}
Map<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreUrls, boolean requireActive) {
@@ -701,19 +670,18 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
return result.get();
}
- List<ZkNodeProps> addReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+ AddReplicaCmd.Response addReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
- AddReplicaCmd.Response response = ((AddReplicaCmd) commandMap.get(ADDREPLICA)).addReplica(clusterState, message, results, onComplete);
- if (response == null) return null;
- return response.responseProps;
+ AddReplicaCmd.Response response = commandMap.get(ADDREPLICA).call(clusterState, message, results);
+
+ return response;
}
- AddReplicaCmd.Response addReplicaWithResp(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+ AddReplicaCmd.Response addReplicaWithResp(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
- AddReplicaCmd.Response response = ((AddReplicaCmd) commandMap.get(ADDREPLICA)).addReplica(clusterState, message, results, onComplete);
- if (response == null) return null;
+ AddReplicaCmd.Response response = ((AddReplicaCmd) commandMap.get(ADDREPLICA)).call(clusterState, message, results);
return response;
}
@@ -1004,25 +972,14 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
@Override
public void close() throws IOException {
this.isClosed = true;
- try {
- cloudManager.close();
- } catch (NullPointerException e) {
- // okay
- } finally {
- latches.forEach(countDownLatch -> countDownLatch.countDown());
- latches.clear();
-
- if (tpe != null) {
- if (!tpe.isShutdown()) {
- tpe.shutdownNow();
- try {
- tpe.awaitTermination(10, TimeUnit.SECONDS);
- } catch (InterruptedException e) {
- ParWork.propagateInterrupt(e);
- }
- }
+
+ if (tpe != null) {
+ if (!tpe.isShutdown()) {
+ ExecutorUtil.shutdownAndAwaitTermination(tpe);
}
}
+ latches.forEach(countDownLatch -> countDownLatch.countDown());
+ latches.clear();
// assert ObjectReleaseTracker.release(this);
}
@@ -1033,7 +990,11 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
protected interface Cmd {
- Runnable call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception;
+ AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception;
+ }
+
+ protected interface Finalize {
+ AddReplicaCmd.Response call() throws Exception;
}
/*
@@ -1082,7 +1043,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
// For thread safety, only simple clone the ModifiableSolrParams
ModifiableSolrParams cloneParams = new ModifiableSolrParams();
cloneParams.add(params);
- cloneParams.set(CoreAdminParams.CORE, replica.getStr(ZkStateReader.CORE_NAME_PROP));
+ cloneParams.set(CoreAdminParams.CORE, replica.getName());
sendShardRequest(replica.getStr(ZkStateReader.NODE_NAME_PROP), cloneParams, shardHandler);
} else {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java
index 22792ed..d97490b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java
@@ -25,7 +25,6 @@ import java.util.List;
import java.util.Map;
import org.apache.solr.cloud.OverseerNodePrioritizer;
-import org.apache.solr.common.ParWork;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
@@ -58,7 +57,7 @@ public class OverseerRoleCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
- public Runnable call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
SolrZkClient zkClient = zkStateReader.getZkClient();
Map roles = null;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java
index f0bd7e7..a240809 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java
@@ -22,6 +22,7 @@ import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
+import java.util.stream.Collectors;
import com.codahale.metrics.Timer;
import org.apache.solr.cloud.OverseerTaskProcessor;
@@ -46,7 +47,7 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings("unchecked")
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
String leaderNode = OverseerTaskProcessor.getLeaderNode(zkStateReader.getZkClient());
results.add("leader", leaderNode);
@@ -54,9 +55,6 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
zkStateReader.getZkClient().getData("/overseer/queue",null, stat);
results.add("overseer_queue_size", stat.getNumChildren());
stat = new Stat();
- zkStateReader.getZkClient().getData("/overseer/queue-work",null, stat);
- results.add("overseer_work_queue_size", stat.getNumChildren());
- stat = new Stat();
zkStateReader.getZkClient().getData("/overseer/collection-queue-work",null, stat);
results.add("overseer_collection_queue_size", stat.getNumChildren());
@@ -67,8 +65,6 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings({"rawtypes"})
NamedList stateUpdateQueueStats = new NamedList();
@SuppressWarnings({"rawtypes"})
- NamedList workQueueStats = new NamedList();
- @SuppressWarnings({"rawtypes"})
NamedList collectionQueueStats = new NamedList();
Stats stats = ocmh.stats;
for (Map.Entry<String, Stats.Stat> entry : stats.getStats().entrySet()) {
@@ -93,8 +89,6 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
}
} else if (key.startsWith("/overseer/queue_")) {
stateUpdateQueueStats.add(key.substring(16), lst);
- } else if (key.startsWith("/overseer/queue-work_")) {
- workQueueStats.add(key.substring(21), lst);
} else if (key.startsWith("/overseer/collection-queue-work_")) {
collectionQueueStats.add(key.substring(32), lst);
} else {
@@ -111,9 +105,13 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
results.add("overseer_operations", overseerStats);
results.add("collection_operations", collectionStats);
results.add("overseer_queue", stateUpdateQueueStats);
- results.add("overseer_internal_queue", workQueueStats);
results.add("collection_queue", collectionQueueStats);
- return null;
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+ response.clusterState = clusterState;
+
+ return response;
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
index fd3516a..8d61ef2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
@@ -167,7 +167,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
@Override
@SuppressWarnings({"unchecked"})
- public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
log.debug("*** called: {}", message);
@@ -618,8 +618,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
}
// build a baseUrl of the replica
for (Replica r : coll.getReplicas()) {
- if (replicaName.equals(r.getCoreName())) {
- return r.getBaseUrl() + "/" + r.getCoreName();
+ if (replicaName.equals(r.getName())) {
+ return r.getBaseUrl() + "/" + r.getName();
}
}
return null;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java
index b5d61f5..bc7d886 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java
@@ -44,7 +44,7 @@ public class RenameCmd implements OverseerCollectionMessageHandler.Cmd {
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String extCollectionName = message.getStr(CoreAdminParams.NAME);
String target = message.getStr(CollectionAdminParams.TARGET);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
index da99e84..20cce88 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
@@ -27,6 +27,8 @@ import java.util.Locale;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
import org.apache.solr.cloud.ActiveReplicaWatcher;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrCloseableLatch;
@@ -61,7 +63,7 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked"})
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
String source = message.getStr(CollectionParams.SOURCE_NODE, message.getStr("source"));
String target = message.getStr(CollectionParams.TARGET_NODE, message.getStr("target"));
@@ -92,63 +94,39 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
// map of collectionName_coreNodeName to watchers
- Map<String, CollectionStateWatcher> watchers = new HashMap<>();
+ Map<String,CollectionStateWatcher> watchers = new HashMap<>();
List<ZkNodeProps> createdReplicas = new ArrayList<>();
AtomicBoolean anyOneFailed = new AtomicBoolean(false);
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(sourceReplicas.size(), ocmh);
SolrCloseableLatch replicasToRecover = new SolrCloseableLatch(numLeaders, ocmh);
- try {
- for (ZkNodeProps sourceReplica : sourceReplicas) {
- @SuppressWarnings({"rawtypes"})
- NamedList nl = new NamedList();
- String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
- if (log.isInfoEnabled()) {
- log.info("Going to create replica for collection={} shard={} on node={}", sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
- }
- String targetNode = target;
- if (targetNode == null) {
- Replica.Type replicaType = Replica.Type.get(sourceReplica.getStr(ZkStateReader.REPLICA_TYPE));
- int numNrtReplicas = replicaType == Replica.Type.NRT ? 1 : 0;
- int numTlogReplicas = replicaType == Replica.Type.TLOG ? 1 : 0;
- int numPullReplicas = replicaType == Replica.Type.PULL ? 1 : 0;
- Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
- .forCollection(sourceCollection)
- .forShard(Collections.singletonList(sourceReplica.getStr(SHARD_ID_PROP)))
- .assignNrtReplicas(numNrtReplicas)
- .assignTlogReplicas(numTlogReplicas)
- .assignPullReplicas(numPullReplicas)
- .onNodes(new ArrayList<>(ocmh.cloudManager.getClusterStateProvider().getLiveNodes()))
- .build();
- Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
- Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, clusterState.getCollection(sourceCollection));
- targetNode = assignStrategy.assign(ocmh.cloudManager, assignRequest).get(0).node;
- }
- ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
- if (async != null) msg.getProperties().put(ASYNC, async);
- final ZkNodeProps addedReplica = ocmh.addReplica(clusterState,
- msg, nl, () -> {
- countDownLatch.countDown();
- if (nl.get("failure") != null) {
- String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
- " on node=%s", sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
- log.warn(errorString);
- // one replica creation failed. Make the best attempt to
- // delete all the replicas created so far in the target
- // and exit
- synchronized (results) {
- results.add("failure", errorString);
- anyOneFailed.set(true);
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("Successfully created replica for collection={} shard={} on node={}",
- sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
- }
- }
- }).get(0);
-
+ List<Runnable> runners = new ArrayList<>();
+ for (ZkNodeProps sourceReplica : sourceReplicas) {
+ @SuppressWarnings({"rawtypes"}) NamedList nl = new NamedList();
+ String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
+ if (log.isInfoEnabled()) {
+ log.info("Going to create replica for collection={} shard={} on node={}", sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
+ }
+ String targetNode = target;
+ if (targetNode == null) {
+ Replica.Type replicaType = Replica.Type.get(sourceReplica.getStr(ZkStateReader.REPLICA_TYPE));
+ int numNrtReplicas = replicaType == Replica.Type.NRT ? 1 : 0;
+ int numTlogReplicas = replicaType == Replica.Type.TLOG ? 1 : 0;
+ int numPullReplicas = replicaType == Replica.Type.PULL ? 1 : 0;
+ Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder().forCollection(sourceCollection).forShard(Collections.singletonList(sourceReplica.getStr(SHARD_ID_PROP)))
+ .assignNrtReplicas(numNrtReplicas).assignTlogReplicas(numTlogReplicas).assignPullReplicas(numPullReplicas)
+ .onNodes(new ArrayList<>(ocmh.cloudManager.getClusterStateProvider().getLiveNodes())).build();
+ Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
+ Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
+ targetNode = assignStrategy.assign(ocmh.cloudManager, assignRequest).get(0).node;
+ }
+ ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
+ if (async != null) msg.getProperties().put(ASYNC, async);
+ AddReplicaCmd.Response response = ocmh.addReplica(clusterState, msg, nl);
+ clusterState = response.clusterState;
+ Runnable runner = () -> {
+ final ZkNodeProps addedReplica = response.responseProps.get(0);
if (addedReplica != null) {
createdReplicas.add(addedReplica);
if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
@@ -158,11 +136,9 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
String key = collectionName + "_" + replicaName;
CollectionStateWatcher watcher;
if (waitForFinalState) {
- watcher = new ActiveReplicaWatcher(collectionName, Collections.singletonList(replicaName),
- null, replicasToRecover);
+ watcher = new ActiveReplicaWatcher(collectionName, Collections.singletonList(replicaName), null, replicasToRecover);
} else {
- watcher = new LeaderRecoveryWatcher(collectionName, shardName, replicaName,
- addedReplica.getStr(ZkStateReader.CORE_NAME_PROP), replicasToRecover);
+ watcher = new LeaderRecoveryWatcher(collectionName, shardName, replicaName, addedReplica.getStr(ZkStateReader.CORE_NAME_PROP), replicasToRecover);
}
watchers.put(key, watcher);
log.debug("--- adding {}, {}", key, watcher);
@@ -171,66 +147,90 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
log.debug("--- not waiting for {}", addedReplica);
}
}
+ };
+ runners.add(runner);
+ }
+
+
+
+ AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+ response.results = results;
+ response.clusterState = clusterState;
+
+ int finalNumLeaders = numLeaders;
+ response.asyncFinalRunner = () -> {
+ for (Runnable runner : runners) {
+ runner.run();
}
log.debug("Waiting for replicas to be added");
- if (!countDownLatch.await(timeout, TimeUnit.SECONDS)) {
- log.info("Timed out waiting for replicas to be added");
- anyOneFailed.set(true);
- } else {
- log.debug("Finished waiting for replicas to be added");
+ try {
+ if (!countDownLatch.await(timeout, TimeUnit.SECONDS)) {
+ log.info("Timed out waiting for replicas to be added");
+ anyOneFailed.set(true);
+ } else {
+ log.debug("Finished waiting for replicas to be added");
+ }
+
+ // now wait for leader replicas to recover
+ log.debug("Waiting for {} leader replicas to recover", finalNumLeaders);
+ if (!replicasToRecover.await(timeout, TimeUnit.SECONDS)) {
+ if (log.isInfoEnabled()) {
+ log.info("Timed out waiting for {} leader replicas to recover", replicasToRecover.getCount());
+ }
+ anyOneFailed.set(true);
+ } else {
+ log.debug("Finished waiting for leader replicas to recover");
+ }
+ } catch (InterruptedException e) {
+ ParWork.propagateInterrupt(e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
- } finally {
- }
- // now wait for leader replicas to recover
- log.debug("Waiting for {} leader replicas to recover", numLeaders);
- if (!replicasToRecover.await(timeout, TimeUnit.SECONDS)) {
- if (log.isInfoEnabled()) {
- log.info("Timed out waiting for {} leader replicas to recover", replicasToRecover.getCount());
+ // remove the watchers, we're done either way
+ for (Map.Entry<String,CollectionStateWatcher> e : watchers.entrySet()) {
+ zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
}
- anyOneFailed.set(true);
- } else {
- log.debug("Finished waiting for leader replicas to recover");
- }
- // remove the watchers, we're done either way
- for (Map.Entry<String, CollectionStateWatcher> e : watchers.entrySet()) {
- zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
- }
- if (anyOneFailed.get()) {
- log.info("Failed to create some replicas. Cleaning up all replicas on target node");
- SolrCloseableLatch cleanupLatch = new SolrCloseableLatch(createdReplicas.size(), ocmh);
- for (ZkNodeProps createdReplica : createdReplicas) {
- @SuppressWarnings({"rawtypes"})
- NamedList deleteResult = new NamedList();
- try {
- ocmh.deleteReplica(zkStateReader.getClusterState(), createdReplica.plus("parallel", "true"), deleteResult, () -> {
+ if (anyOneFailed.get()) {
+ log.info("Failed to create some replicas. Cleaning up all replicas on target node");
+ SolrCloseableLatch cleanupLatch = new SolrCloseableLatch(createdReplicas.size(), ocmh);
+ for (ZkNodeProps createdReplica : createdReplicas) {
+ @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
+ try {
+ // nocommit - return results from deleteReplica cmd, update clusterstate
+ ocmh.deleteReplica(zkStateReader.getClusterState(), createdReplica.plus("parallel", "true"), deleteResult);
+ } catch (KeeperException e) {
cleanupLatch.countDown();
- if (deleteResult.get("failure") != null) {
- synchronized (results) {
- results.add("failure", "Could not cleanup, because of : " + deleteResult.get("failure"));
- }
- }
- });
- } catch (KeeperException e) {
- cleanupLatch.countDown();
- log.warn("Error deleting replica ", e);
- } catch (Exception e) {
+ log.warn("Error deleting replica ", e);
+ } catch (Exception e) {
+ ParWork.propagateInterrupt(e);
+ log.warn("Error deleting replica ", e);
+ cleanupLatch.countDown();
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+ }
+ try {
+ cleanupLatch.await(5, TimeUnit.MINUTES);
+ } catch (InterruptedException e) {
ParWork.propagateInterrupt(e);
- log.warn("Error deleting replica ", e);
- cleanupLatch.countDown();
- throw e;
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
- cleanupLatch.await(5, TimeUnit.MINUTES);
- return null;
- }
+ // we have reached this far means all replicas could be recreated
+ //now cleanup the replicas in the source node
+ try {
+ DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ocmh, source, null);
+ } catch (InterruptedException e) {
+ ParWork.propagateInterrupt(e);
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+ results.add("success", "REPLACENODE action completed successfully from : " + source + " to : " + target);
+ AddReplicaCmd.Response resp = new AddReplicaCmd.Response();
+ return resp;
+
+ };
- // we have reached this far means all replicas could be recreated
- //now cleanup the replicas in the source node
- DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ocmh, source, null);
- results.add("success", "REPLACENODE action completed successfully from : " + source + " to : " + target);
- return null;
+ return response;
}
static List<ZkNodeProps> getReplicasOfNode(String source, ClusterState state) {
@@ -242,8 +242,7 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
ZkNodeProps props = new ZkNodeProps(
COLLECTION_PROP, e.getKey(),
SHARD_ID_PROP, slice.getName(),
- ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
- ZkStateReader.REPLICA_PROP, replica.getName(),
+ ZkStateReader.CORE_NAME_PROP, replica.getName(),
ZkStateReader.REPLICA_TYPE, replica.getType().name(),
ZkStateReader.LEADER_PROP, String.valueOf(replica.equals(slice.getLeader())),
CoreAdminParams.NODE, source);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
index e2b01dc..219f304 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
@@ -87,7 +87,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
- public Runnable call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
// TODO maybe we can inherit createCollection's options/code
String restoreCollectionName = message.getStr(COLLECTION_PROP);
@@ -242,8 +242,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
.assignPullReplicas(numPullReplicas)
.onNodes(nodeList)
.build();
+
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
- Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, restoreCollection);
+ Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
CountDownLatch countDownLatch = new CountDownLatch(restoreCollection.getSlices().size());
@@ -284,25 +285,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
ocmh.addPropertyParams(message, propMap);
final NamedList addReplicaResult = new NamedList();
- ocmh.addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult, () -> {
- Object addResultFailure = addReplicaResult.get("failure");
- if (addResultFailure != null) {
- SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
- if (failure == null) {
- failure = new SimpleOrderedMap();
- results.add("failure", failure);
- }
- failure.addAll((NamedList) addResultFailure);
- } else {
- SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
- if (success == null) {
- success = new SimpleOrderedMap();
- results.add("success", success);
- }
- success.addAll((NamedList) addReplicaResult.get("success"));
- }
- countDownLatch.countDown();
- });
+ ocmh.addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult);
}
boolean allIsDone = countDownLatch.await(1, TimeUnit.HOURS);
@@ -339,17 +322,17 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
for (Slice s : restoreCollection.getSlices()) {
for (Replica r : s.getReplicas()) {
String nodeName = r.getNodeName();
- String coreNodeName = r.getCoreName();
+ String coreName = r.getName();
Replica.State stateRep = r.getState();
if (log.isDebugEnabled()) {
- log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}", nodeName, coreNodeName,
+ log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreName={}, state={}", nodeName, coreName,
stateRep.name());
}
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
- params.set(CoreAdminParams.NAME, coreNodeName);
+ params.set(CoreAdminParams.NAME, coreName);
shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
}
@@ -425,7 +408,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
ocmh.addPropertyParams(message, propMap);
- ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
+ ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results);
}
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
index 1e82000..c609086 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
@@ -47,7 +47,7 @@ public class SetAliasPropCmd implements Cmd {
}
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
String aliasName = message.getStr(NAME);
final ZkStateReader.AliasesManager aliasesManager = messageHandler.zkStateReader.aliasesManager;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
index cdc2555..24dac32 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
@@ -27,9 +27,9 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@@ -38,10 +38,13 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.ReplicaInfo;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.CollectionMutator;
import org.apache.solr.cloud.overseer.OverseerAction;
+import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.ParWork;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@@ -67,6 +70,7 @@ import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.update.SolrIndexSplitter;
import org.apache.solr.util.RTimerTree;
import org.apache.solr.util.TestInjection;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -98,13 +102,12 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
@SuppressWarnings("unchecked")
@Override
- public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
- split(state, message,(NamedList<Object>) results);
- return null;
+ public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+ return split(state, message,(NamedList<Object>) results);
}
@SuppressWarnings({"rawtypes"})
- public boolean split(ClusterState clusterState, ZkNodeProps message, NamedList<Object> results) throws Exception {
+ public AddReplicaCmd.Response split(ClusterState clusterState, ZkNodeProps message, NamedList<Object> results) throws Exception {
final String asyncId = message.getStr(ASYNC);
boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
@@ -126,7 +129,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
collectionName = extCollectionName;
}
- log.debug("Split shard invoked: {}", message);
+ log.info("Split shard invoked: {} cs={}", message, clusterState);
ZkStateReader zkStateReader = ocmh.zkStateReader;
AtomicReference<String> slice = new AtomicReference<>();
slice.set(message.getStr(ZkStateReader.SHARD_ID_PROP));
@@ -134,13 +137,12 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
RTimerTree timings = new RTimerTree();
String splitKey = message.getStr("split.key");
- DocCollection collection = clusterState.getCollection(collectionName);
- Slice parentSlice = getParentSlice(zkStateReader, collectionName, slice, splitKey);
- if (parentSlice.getState() != Slice.State.ACTIVE) {
- throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Parent slice is not active: " +
- collectionName + "/ " + parentSlice.getName() + ", state=" + parentSlice.getState());
- }
+ Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
+// if (parentSlice.getState() != Slice.State.ACTIVE) {
+// throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Parent slice is not active: " +
+// collectionName + "/ " + parentSlice.getName() + ", state=" + parentSlice.getState());
+// }
// find the leader for the shard
Replica parentShardLeader;
@@ -197,6 +199,8 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
List<Map<String, Object>> replicas = new ArrayList<>((repFactor - 1) * 2);
+ Map<String, Object> replicaToPosition = new HashMap<>(replicas.size());
+
@SuppressWarnings("deprecation")
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
@@ -208,7 +212,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.SPLIT.toString());
params.set(CoreAdminParams.GET_RANGES, "true");
params.set(CommonAdminParams.SPLIT_METHOD, splitMethod.toLower());
- params.set(CoreAdminParams.CORE, parentShardLeader.getStr("core"));
+ params.set(CoreAdminParams.CORE, parentShardLeader.getName());
// Only 2 is currently supported
// int numSubShards = message.getInt(NUM_SUB_SHARDS, DEFAULT_NUM_SUB_SHARDS);
// params.set(NUM_SUB_SHARDS, Integer.toString(numSubShards));
@@ -239,8 +243,8 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
t = timings.sub("fillRanges");
-
- String rangesStr = fillRanges(ocmh.cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames, firstNrtReplica);
+ DocCollection collection = clusterState.getCollection(collectionName);
+ String rangesStr = fillRanges(message, collection, parentSlice, subRanges, subSlices, subShardNames, firstNrtReplica);
t.stop();
boolean oldShardsDeleted = false;
@@ -250,7 +254,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
final Slice.State state = oSlice.getState();
if (state == Slice.State.ACTIVE) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
- "Sub-shard: " + subSlice + " exists in active state. Aborting split shard.");
+ "Sub-shard: " + subSlice + " exists in active state. Aborting split shard. Parent=" + parentSlice.getName());
} else {
// delete the shards
log.info("Sub-shard: {} already exists therefore requesting its deletion", subSlice);
@@ -260,7 +264,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(SHARD_ID_PROP, subSlice);
ZkNodeProps m = new ZkNodeProps(propMap);
try {
- ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList());
+ clusterState = ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList()).clusterState;
} catch (Exception e) {
ParWork.propagateInterrupt(e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to delete already existing sub shard: " + subSlice,
@@ -275,14 +279,13 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
if (oldShardsDeleted) {
// refresh the locally cached cluster state
// we know we have the latest because otherwise deleteshard would have failed
- clusterState = zkStateReader.getClusterState();
collection = clusterState.getCollection(collectionName);
}
String nodeName = parentShardLeader.getNodeName();
t = timings.sub("createSubSlicesAndLeadersInState");
- List<Future> firstReplicaFutures = new ArrayList<>();
+ List<OverseerCollectionMessageHandler.Finalize> firstReplicaFutures = new ArrayList<>();
Set<Runnable> firstReplicaRunAfters = ConcurrentHashMap.newKeySet();
for (int i = 0; i < subRanges.size(); i++) {
String subSlice = subSlices.get(i);
@@ -301,13 +304,10 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put("shard_parent_node", nodeName);
propMap.put("shard_parent_zk_session", leaderZnodeStat.getEphemeralOwner());
- ocmh.overseer.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
-
+ //ocmh.overseer.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
+ clusterState = new CollectionMutator(ocmh.cloudManager).createShard(clusterState, new ZkNodeProps(propMap));
// wait until we are able to see the new shard in cluster state and refresh the local view of the cluster state
- ocmh.waitForNewShard(collectionName, subSlice);
-
- // refresh cluster state
- clusterState = zkStateReader.getClusterState();
+ //ocmh.waitForNewShard(collectionName, subSlice);
log.debug("Adding first replica {} as part of slice {} of collection {} on {}"
, subShardName, subSlice, collectionName, nodeName);
@@ -329,34 +329,36 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
-
- Map<String,Object> finalPropMap = propMap;
- ClusterState finalClusterState1 = clusterState;
- Future<?> future = ocmh.tpe.submit(() -> {
- AddReplicaCmd.Response response = null;
- try {
- response = ocmh.addReplicaWithResp(finalClusterState1, new ZkNodeProps(finalPropMap), results, null);
- } catch (Exception e) {
- log.error("", e);
- }
- if (response != null && response.asyncFinalRunner != null) {
- firstReplicaRunAfters.add(response.asyncFinalRunner);
- }
- });
- firstReplicaFutures.add(future);
+ AddReplicaCmd.Response resp = ocmh.addReplicaWithResp(clusterState, new ZkNodeProps(propMap), results);
+ clusterState = resp.clusterState;
+ firstReplicaFutures.add(resp.asyncFinalRunner);
+// Map<String,Object> finalPropMap = propMap;
+// ClusterState finalClusterState1 = clusterState;
+// Future<?> future = ocmh.tpe.submit(() -> {
+// AddReplicaCmd.Response response = null;
+// try {
+// response = ocmh.addReplicaWithResp(finalClusterState1, new ZkNodeProps(finalPropMap), results, null);
+// } catch (Exception e) {
+// log.error("", e);
+// }
+//// if (response != null && response.asyncFinalRunner != null) {
+//// firstReplicaRunAfters.add(response.asyncFinalRunner);
+//// }
+// });
+// firstReplicaFutures.add(future);
}
+ ocmh.overseer.getZkStateWriter().enqueueUpdate(clusterState, false);
+ ocmh.overseer.writePendingUpdates();
firstReplicaFutures.forEach(future -> {
try {
- future.get();
- } catch (InterruptedException e) {
- log.error("", e);
- } catch (ExecutionException e) {
- log.error("", e);
+ future.call();
+ } catch (Exception e) {
+ log.error("Exception waiting for created replica", e);
}
});
- firstReplicaRunAfters.forEach(runnable -> runnable.run());
+ // firstReplicaRunAfters.forEach(runnable -> runnable.run());
{
final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
@@ -370,15 +372,16 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
for (String subShardName : subShardNames) {
// wait for parent leader to acknowledge the sub-shard core
- log.debug("Asking parent leader to wait for: {} to be alive on: {}", subShardName, nodeName);
- String coreNodeName = OverseerCollectionMessageHandler.waitForCoreNodeName(zkStateReader, collectionName, nodeName, subShardName);
+ log.info("Asking parent leader to wait for: {} to be alive on: {}", subShardName, nodeName);
+
CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
cmd.setCoreName(subShardName);
cmd.setNodeName(nodeName);
- cmd.setCoreNodeName(coreNodeName);
cmd.setState(Replica.State.ACTIVE);
cmd.setCheckLive(true);
cmd.setOnlyIfLeader(true);
+ cmd.setCollection(collectionName);
+
ModifiableSolrParams p = new ModifiableSolrParams(cmd.getParams());
shardRequestTracker.sendShardRequest(nodeName, p, shardHandler);
@@ -401,7 +404,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.SPLIT.toString());
params.set(CommonAdminParams.SPLIT_METHOD, splitMethod.toLower());
- params.set(CoreAdminParams.CORE, parentShardLeader.getStr("core"));
+ params.set(CoreAdminParams.CORE, parentShardLeader.getName());
for (int i = 0; i < subShardNames.size(); i++) {
String subShardName = subShardNames.get(i);
params.add(CoreAdminParams.TARGET_CORE, subShardName);
@@ -423,30 +426,31 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
log.debug("Index on shard: {} split into {} successfully", nodeName, subShardNames.size());
}
- t = timings.sub("applyBufferedUpdates");
- // apply buffered updates on sub-shards
- {
- final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
-
- for (int i = 0; i < subShardNames.size(); i++) {
- String subShardName = subShardNames.get(i);
-
- log.debug("Applying buffered updates on : {}", subShardName);
-
- params = new ModifiableSolrParams();
- params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
- params.set(CoreAdminParams.NAME, subShardName);
-
- shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
- }
-
- String msgOnError = "SPLITSHARD failed while asking sub shard leaders to apply buffered updates";
- shardRequestTracker.processResponses(results, shardHandler, true, msgOnError);
- handleFailureOnAsyncRequest(results, msgOnError);
- }
- t.stop();
-
- log.debug("Successfully applied buffered updates on : {}", subShardNames);
+ // nocommit - where do we enter buffering state??
+// t = timings.sub("applyBufferedUpdates");
+// // apply buffered updates on sub-shards
+// {
+// final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+//
+// for (int i = 0; i < subShardNames.size(); i++) {
+// String subShardName = subShardNames.get(i);
+//
+// log.debug("Applying buffered updates on : {}", subShardName);
+//
+// params = new ModifiableSolrParams();
+// params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
+// params.set(CoreAdminParams.NAME, subShardName);
+//
+// shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
+// }
+//
+// String msgOnError = "SPLITSHARD failed while asking sub shard leaders to apply buffered updates";
+// shardRequestTracker.processResponses(results, shardHandler, true, msgOnError);
+// handleFailureOnAsyncRequest(results, msgOnError);
+// }
+// t.stop();
+//
+// log.debug("Successfully applied buffered updates on : {}", subShardNames);
// Replica creation for the new Slices
// replica placement is controlled by the autoscaling policy framework
@@ -480,7 +484,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
.onNodes(new ArrayList<>(clusterState.getLiveNodes()))
.build();
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
- Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, collection);
+ Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
t.stop();
@@ -488,7 +492,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
for (ReplicaPosition replicaPosition : replicaPositions) {
String sliceName = replicaPosition.shard;
String subShardNodeName = replicaPosition.node;
- String solrCoreName = Assign.buildSolrCoreName(collectionName, sliceName, replicaPosition.type, replicaPosition.index);
+ String solrCoreName = Assign.buildSolrCoreName(collection, collectionName, sliceName, replicaPosition.type);
log.debug("Creating replica shard {} as part of slice {} of collection {} on {}"
, solrCoreName, sliceName, collectionName, subShardNodeName);
@@ -497,16 +501,16 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
// avoid a race condition where Overseer may prematurely activate the new sub-slices (and deactivate
// the parent slice) before all new replicas are added. This situation may lead to a loss of performance
// because the new shards will be activated with possibly many fewer replicas.
- ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
- ZkStateReader.COLLECTION_PROP, collectionName,
- ZkStateReader.SHARD_ID_PROP, sliceName,
- ZkStateReader.CORE_NAME_PROP, solrCoreName,
- ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
- ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
- ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
- ZkStateReader.NODE_NAME_PROP, subShardNodeName,
- CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
- ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
+// ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
+// ZkStateReader.COLLECTION_PROP, collectionName,
+// ZkStateReader.SHARD_ID_PROP, sliceName,
+// ZkStateReader.CORE_NAME_PROP, solrCoreName,
+// ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
+// ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
+// ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
+// ZkStateReader.NODE_NAME_PROP, subShardNodeName,
+// CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
+// ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
@@ -531,35 +535,36 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
propMap.put(CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
replicas.add(propMap);
+ replicaToPosition.put(solrCoreName, replicaPosition);
}
t.stop();
assert TestInjection.injectSplitFailureBeforeReplicaCreation();
- long ephemeralOwner = leaderZnodeStat.getEphemeralOwner();
- // compare against the ephemeralOwner of the parent leader node
- leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNodeName(), null);
- if (leaderZnodeStat == null || ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
- // put sub-shards in recovery_failed state
-
- Map<String, Object> propMap = new HashMap<>();
- propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
- for (String subSlice : subSlices) {
- propMap.put(subSlice, Slice.State.RECOVERY_FAILED.toString());
- }
- propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
- ZkNodeProps m = new ZkNodeProps(propMap);
- ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
-
- if (leaderZnodeStat == null) {
- // the leader is not live anymore, fail the split!
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNodeName() + " is not live anymore!");
- } else if (ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
- // there's a new leader, fail the split!
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "The zk session id for the shard leader node: " + parentShardLeader.getNodeName() + " has changed from "
- + ephemeralOwner + " to " + leaderZnodeStat.getEphemeralOwner() + ". This can cause data loss so we must abort the split");
- }
- }
+// long ephemeralOwner = leaderZnodeStat.getEphemeralOwner();
+// // compare against the ephemeralOwner of the parent leader node
+// leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNodeName(), null);
+// if (leaderZnodeStat == null || ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
+// // put sub-shards in recovery_failed state
+//
+// Map<String, Object> propMap = new HashMap<>();
+// propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
+// for (String subSlice : subSlices) {
+// propMap.put(subSlice, Slice.State.RECOVERY_FAILED.toString());
+// }
+// propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
+// ZkNodeProps m = new ZkNodeProps(propMap);
+// ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
+//
+// if (leaderZnodeStat == null) {
+// // the leader is not live anymore, fail the split!
+// throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNodeName() + " is not live anymore!");
+// } else if (ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
+// // there's a new leader, fail the split!
+// throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+// "The zk session id for the shard leader node: " + parentShardLeader.getNodeName() + " has changed from "
+// + ephemeralOwner + " to " + leaderZnodeStat.getEphemeralOwner() + ". This can cause data loss so we must abort the split");
+// }
+// }
// we must set the slice state into recovery before actually creating the replica cores
// this ensures that the logic inside ReplicaMutator to update sub-shard state to 'active'
@@ -599,24 +604,27 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
List<Future> replicaFutures = new ArrayList<>();
Set<Runnable> replicaRunAfters = ConcurrentHashMap.newKeySet();
+// for (Map<String, Object> replica : replicas) {
+// ocmh.addReplica(clusterState, new ZkNodeProps(replica), results);
+// }
// now actually create replica cores on sub shard nodes
- for (Map<String, Object> replica : replicas) {
- ClusterState finalClusterState = clusterState;
- Future<?> future = ocmh.tpe.submit(() -> {
- AddReplicaCmd.Response response = null;
- try {
- response = ocmh.addReplicaWithResp(finalClusterState, new ZkNodeProps(replica), results, null);
- } catch (Exception e) {
- log.error("", e);
- }
- if (response != null && response.asyncFinalRunner != null) {
- replicaRunAfters.add(response.asyncFinalRunner);
- }
- });
-
- replicaFutures.add(future);
- }
+// for (Map<String, Object> replica : replicas) {
+// ClusterState finalClusterState = clusterState;
+// Future<?> future = ocmh.overseer.getTaskExecutor().submit(() -> {
+// AddReplicaCmd.Response response = null;
+// try {
+// response = ocmh.addReplicaWithResp(finalClusterState, new ZkNodeProps(replica), results, null);
+// } catch (Exception e) {
+// log.error("", e);
+// }
+// if (response != null && response.asyncFinalRunner != null) {
+// replicaRunAfters.add(response.asyncFinalRunner);
+// }
+// });
+//
+// replicaFutures.add(future);
+// }
assert TestInjection.injectSplitFailureAfterReplicaCreation();
@@ -628,15 +636,15 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
}
t.stop();
- replicaFutures.forEach(future -> {
- try {
- future.get();
- } catch (InterruptedException e) {
- log.error("", e);
- } catch (ExecutionException e) {
- log.error("", e);
- }
- });
+// replicaFutures.forEach(future -> {
+// try {
+// future.get();
+// } catch (InterruptedException e) {
+// log.error("", e);
+// } catch (ExecutionException e) {
+// log.error("", e);
+// }
+// });
log.info("Successfully created all replica shards for all sub-slices {}", subSlices);
... 10276 lines suppressed ...