From cc0c111949d5039a0c7cb67cad55c63e2f761298 Mon Sep 17 00:00:00 2001
From: Ishan Chattopadhyaya
Date: Mon, 20 Jul 2020 17:47:01 +0530
Subject: [PATCH] SOLR-14656: Removing Autoscaling Framework
The following were removed:
* Autoscaling policy, triggers etc.
* withCollection handling
* UTILIZENODE command
* Sim framework
* Suggestions tab in UI
* Reference guide pages for autoscaling
* autoAddReplicas feature
* UTILIZENODE
---
solr/CHANGES.txt | 2 +
solr/bin/solr | 7 +-
solr/bin/solr.cmd | 10 +-
.../java/org/apache/solr/cloud/CloudUtil.java | 1 -
.../java/org/apache/solr/cloud/Overseer.java | 13 +-
.../org/apache/solr/cloud/ZkController.java | 41 -
.../cloud/api/collections/AddReplicaCmd.java | 17 +-
.../solr/cloud/api/collections/Assign.java | 139 +-
.../api/collections/CreateCollectionCmd.java | 20 +-
.../OverseerCollectionMessageHandler.java | 8 +-
.../api/collections/ReindexCollectionCmd.java | 5 +-
.../cloud/api/collections/ReplaceNodeCmd.java | 7 -
.../cloud/api/collections/RestoreCmd.java | 410 +-
.../cloud/api/collections/SplitShardCmd.java | 52 +-
.../cloud/api/collections/UtilizeNodeCmd.java | 139 -
.../solr/cloud/autoscaling/ActionContext.java | 68 -
.../AutoAddReplicasPlanAction.java | 40 -
.../solr/cloud/autoscaling/AutoScaling.java | 246 -
.../cloud/autoscaling/AutoScalingHandler.java | 742 -
.../cloud/autoscaling/ComputePlanAction.java | 355 -
.../cloud/autoscaling/ExecutePlanAction.java | 250 -
.../autoscaling/HttpTriggerListener.java | 164 -
.../InactiveMarkersPlanAction.java | 141 -
.../autoscaling/InactiveShardPlanAction.java | 163 -
.../cloud/autoscaling/IndexSizeTrigger.java | 562 -
.../cloud/autoscaling/LoggingListener.java | 38 -
.../solr/cloud/autoscaling/MetricTrigger.java | 220 -
.../cloud/autoscaling/NodeAddedTrigger.java | 250 -
.../cloud/autoscaling/NodeLostTrigger.java | 240 -
.../autoscaling/OverseerTriggerThread.java | 407 -
.../cloud/autoscaling/ScheduledTrigger.java | 227 -
.../cloud/autoscaling/ScheduledTriggers.java | 913 --
.../cloud/autoscaling/SearchRateTrigger.java | 805 -
.../cloud/autoscaling/SystemLogListener.java | 221 -
.../solr/cloud/autoscaling/TriggerAction.java | 51 -
.../cloud/autoscaling/TriggerActionBase.java | 87 -
.../autoscaling/TriggerActionException.java | 33 -
.../solr/cloud/autoscaling/TriggerBase.java | 300 -
.../solr/cloud/autoscaling/TriggerEvent.java | 315 -
.../cloud/autoscaling/TriggerEventQueue.java | 121 -
.../cloud/autoscaling/TriggerListener.java | 65 -
.../autoscaling/TriggerListenerBase.java | 97 -
.../solr/cloud/autoscaling/TriggerUtils.java | 88 -
.../TriggerValidationException.java | 74 -
.../solr/cloud/autoscaling/package-info.java | 21 -
.../cloud/autoscaling/sim/ActionError.java | 24 -
.../autoscaling/sim/FakeDocIterator.java | 56 -
.../sim/GenericDistributedQueue.java | 601 -
.../sim/GenericDistributedQueueFactory.java | 45 -
.../cloud/autoscaling/sim/LiveNodesSet.java | 113 -
.../sim/NoopDistributedQueueFactory.java | 94 -
.../autoscaling/sim/SimCloudManager.java | 1060 --
.../sim/SimClusterStateProvider.java | 2574 ---
.../sim/SimDistribStateManager.java | 649 -
.../sim/SimDistributedQueueFactory.java | 287 -
.../autoscaling/sim/SimNodeStateProvider.java | 359 -
.../cloud/autoscaling/sim/SimScenario.java | 1129 --
.../solr/cloud/autoscaling/sim/SimUtils.java | 434 -
.../autoscaling/sim/SnapshotCloudManager.java | 260 -
.../sim/SnapshotClusterStateProvider.java | 152 -
.../sim/SnapshotDistribStateManager.java | 228 -
.../sim/SnapshotNodeStateProvider.java | 203 -
.../cloud/autoscaling/sim/package-info.java | 94 -
.../solr/cloud/overseer/ReplicaMutator.java | 2 +-
.../org/apache/solr/core/CoreContainer.java | 12 -
.../apache/solr/core/SolrResourceLoader.java | 3 +-
.../admin/AutoscalingHistoryHandler.java | 166 -
.../handler/admin/CollectionsHandler.java | 12 +-
.../handler/admin/MetricsHistoryHandler.java | 6 +-
.../solr/security/PermissionNameProvider.java | 3 -
.../java/org/apache/solr/util/SolrCLI.java | 390 -
.../solr/simSnapshot/autoscalingState.json | 3923 -----
.../solr/simSnapshot/clusterState.json | 2838 ----
.../solr/simSnapshot/distribState.json | 206 -
.../solr/simSnapshot/managerState.json | 1 -
.../solr/simSnapshot/nodeState.json | 3823 -----
.../solr/simSnapshot/statistics.json | 2029 ---
.../org/apache/solr/cloud/CloudTestUtils.java | 195 -
.../solr/cloud/CollectionsAPISolrJTest.java | 18 +-
.../cloud/MetricsHistoryIntegrationTest.java | 24 +-
.../apache/solr/cloud/MoveReplicaTest.java | 2 -
...rseerCollectionConfigSetProcessorTest.java | 11 +-
.../solr/cloud/ReplaceNodeNoTargetTest.java | 127 -
.../RoutingToNodesWithPropertiesTest.java | 242 -
.../SharedFSAutoReplicaFailoverTest.java | 14 +-
.../solr/cloud/TestClusterProperties.java | 12 -
.../apache/solr/cloud/TestUtilizeNode.java | 184 -
.../apache/solr/cloud/TestWithCollection.java | 611 -
.../AbstractCloudBackupRestoreTestCase.java | 4 -
.../cloud/api/collections/AssignTest.java | 204 -
.../CollectionTooManyReplicasTest.java | 13 +-
.../ConcurrentCreateCollectionTest.java | 292 -
.../AutoAddReplicasIntegrationTest.java | 472 -
.../AutoAddReplicasPlanActionTest.java | 263 -
.../autoscaling/AutoScalingHandlerTest.java | 1114 --
.../solr/cloud/autoscaling/CapturedEvent.java | 65 -
.../autoscaling/ComputePlanActionTest.java | 794 -
.../autoscaling/ExecutePlanActionTest.java | 381 -
.../HdfsAutoAddReplicasIntegrationTest.java | 61 -
.../autoscaling/HttpTriggerListenerTest.java | 208 -
.../IndexSizeTriggerMixedBoundsTest.java | 374 -
.../IndexSizeTriggerSizeEstimationTest.java | 323 -
.../autoscaling/IndexSizeTriggerTest.java | 795 -
.../MetricTriggerIntegrationTest.java | 242 -
.../cloud/autoscaling/MetricTriggerTest.java | 135 -
.../NodeAddedTriggerIntegrationTest.java | 320 -
.../autoscaling/NodeAddedTriggerTest.java | 345 -
.../NodeLostTriggerIntegrationTest.java | 356 -
.../autoscaling/NodeLostTriggerTest.java | 394 -
.../NodeMarkersRegistrationTest.java | 356 -
.../autoscaling/RestoreTriggerStateTest.java | 168 -
.../ScheduledMaintenanceTriggerTest.java | 384 -
.../ScheduledTriggerIntegrationTest.java | 150 -
.../autoscaling/ScheduledTriggerTest.java | 141 -
.../SearchRateTriggerIntegrationTest.java | 747 -
.../autoscaling/SearchRateTriggerTest.java | 357 -
.../autoscaling/SystemLogListenerTest.java | 293 -
.../cloud/autoscaling/TestPolicyCloud.java | 542 -
.../TriggerCooldownIntegrationTest.java | 228 -
.../autoscaling/TriggerEventQueueTest.java | 99 -
.../autoscaling/TriggerIntegrationTest.java | 729 -
.../TriggerSetPropertiesIntegrationTest.java | 275 -
.../autoscaling/sim/SimSolrCloudTestCase.java | 255 -
.../sim/TestSimClusterStateProvider.java | 227 -
.../sim/TestSimComputePlanAction.java | 401 -
.../sim/TestSimDistribStateManager.java | 380 -
.../sim/TestSimDistributedQueue.java | 221 -
.../sim/TestSimExecutePlanAction.java | 224 -
.../sim/TestSimExtremeIndexing.java | 147 -
.../sim/TestSimGenericDistributedQueue.java | 38 -
.../autoscaling/sim/TestSimLargeCluster.java | 870 -
.../sim/TestSimNodeAddedTrigger.java | 331 -
.../sim/TestSimNodeLostTrigger.java | 350 -
.../autoscaling/sim/TestSimPolicyCloud.java | 379 -
.../autoscaling/sim/TestSimScenario.java | 173 -
.../sim/TestSimTriggerIntegration.java | 1551 --
.../cloud/autoscaling/sim/TestSimUtils.java | 95 -
.../sim/TestSnapshotCloudManager.java | 275 -
.../solr/cloud/rule/RuleEngineTest.java | 2 +-
.../org/apache/solr/cloud/rule/RulesTest.java | 67 -
.../solr/handler/V2ApiIntegrationTest.java | 4 +-
.../admin/AutoscalingHistoryHandlerTest.java | 463 -
.../admin/MetricsHistoryHandlerTest.java | 43 +-
.../handler/admin/TestCollectionAPIs.java | 4 +-
.../JWTAuthPluginIntegrationTest.java | 6 +-
.../TestSolrCloudWithHadoopAuthPlugin.java | 6 +-
.../solr/util/TestSolrCLIRunExample.java | 117 -
.../test/org/apache/solr/util/TestUtils.java | 1 -
.../src/cluster-node-management.adoc | 26 +-
.../src/collection-management.adoc | 19 +-
.../src/colocating-collections.adoc | 75 -
.../src/major-changes-in-solr-7.adoc | 2 -
.../src/major-changes-in-solr-8.adoc | 6 +-
.../src/major-changes-in-solr-9.adoc | 10 +
solr/solr-ref-guide/src/metrics-history.adoc | 2 +-
.../src/migrate-to-policy-rule.adoc | 198 -
.../src/replica-management.adoc | 2 -
.../src/rule-based-authorization-plugin.adoc | 15 -
.../src/running-solr-on-hdfs.adoc | 39 -
solr/solr-ref-guide/src/shard-management.adoc | 6 +-
.../src/solr-upgrade-notes.adoc | 4 +-
.../src/solrcloud-autoscaling-api.adoc | 808 -
...lrcloud-autoscaling-auto-add-replicas.adoc | 73 -
...solrcloud-autoscaling-fault-tolerance.adoc | 59 -
.../src/solrcloud-autoscaling-listeners.adoc | 220 -
.../src/solrcloud-autoscaling-overview.adoc | 116 -
...rcloud-autoscaling-policy-preferences.adoc | 638 -
...solrcloud-autoscaling-trigger-actions.adoc | 145 -
.../src/solrcloud-autoscaling-triggers.adoc | 627 -
.../src/solrcloud-autoscaling.adoc | 35 -
solr/solr-ref-guide/src/solrcloud.adoc | 6 +-
.../src/suggestions-screen.adoc | 40 -
...he-solr-administration-user-interface.adoc | 3 +-
solr/solr-ref-guide/src/v2-api.adoc | 7 +-
.../AlreadyExistsException.java | 2 +-
.../BadVersionException.java | 2 +-
.../solrj/cloud/DelegatingCloudManager.java | 89 +
.../cloud/DelegatingClusterStateProvider.java | 130 +
.../solrj/cloud/DistribStateManager.java | 11 -
.../{autoscaling => }/NotEmptyException.java | 2 +-
.../{autoscaling => }/VersionedData.java | 2 +-
.../autoscaling/AddReplicaSuggester.java | 84 -
.../cloud/autoscaling/AutoScalingConfig.java | 616 -
.../client/solrj/cloud/autoscaling/Cell.java | 74 -
.../solrj/cloud/autoscaling/Clause.java | 839 -
.../solrj/cloud/autoscaling/ComputedType.java | 99 -
.../solrj/cloud/autoscaling/Condition.java | 125 -
.../cloud/autoscaling/CoresVariable.java | 127 -
.../autoscaling/DelegatingCloudManager.java | 93 -
.../DelegatingClusterStateProvider.java | 130 -
.../DelegatingDistribStateManager.java | 107 -
.../DelegatingNodeStateProvider.java | 57 -
.../autoscaling/DeleteNodeSuggester.java | 48 -
.../autoscaling/DeleteReplicaSuggester.java | 78 -
.../cloud/autoscaling/FreeDiskVariable.java | 176 -
.../autoscaling/MoveReplicaSuggester.java | 115 -
.../solrj/cloud/autoscaling/NodeVariable.java | 44 -
.../cloud/autoscaling/NoneSuggester.java | 41 -
.../solrj/cloud/autoscaling/Operand.java | 210 -
.../solrj/cloud/autoscaling/Policy.java | 792 -
.../solrj/cloud/autoscaling/PolicyHelper.java | 733 -
.../solrj/cloud/autoscaling/Preference.java | 157 -
.../solrj/cloud/autoscaling/RangeVal.java | 61 -
.../solrj/cloud/autoscaling/ReplicaCount.java | 134 -
.../cloud/autoscaling/ReplicaVariable.java | 176 -
.../client/solrj/cloud/autoscaling/Row.java | 402 -
.../solrj/cloud/autoscaling/SealedClause.java | 29 -
.../autoscaling/SplitShardSuggester.java | 68 -
.../solrj/cloud/autoscaling/Suggester.java | 530 -
.../solrj/cloud/autoscaling/Suggestion.java | 133 -
.../TriggerEventProcessorStage.java | 30 -
.../cloud/autoscaling/TriggerEventType.java | 33 -
.../autoscaling/UnsupportedSuggester.java | 61 -
.../solrj/cloud/autoscaling/Variable.java | 410 -
.../solrj/cloud/autoscaling/VariableBase.java | 213 -
.../solrj/cloud/autoscaling/Violation.java | 182 -
.../autoscaling/WithCollectionVariable.java | 170 -
.../solrj/cloud/autoscaling/package-info.java | 23 -
.../impl/SolrClientNodeStateProvider.java | 85 +-
.../solrj/impl/ZkDistribStateManager.java | 34 +-
.../solrj/request/CollectionAdminRequest.java | 42 -
.../solrj/request/CollectionApiMapping.java | 5 -
.../solr/common/cloud/DocCollection.java | 19 -
.../solr/common/cloud/ZkStateReader.java | 44 -
.../solr/common/params/AutoScalingParams.java | 76 -
.../common/params/CollectionAdminParams.java | 6 -
.../solr/common/params/CollectionParams.java | 1 -
.../solr/common/params/CommonParams.java | 10 +-
.../org/apache/solr/common/util/Utils.java | 2 +-
.../apispec/autoscaling.Commands.json | 199 -
.../apispec/autoscaling.history.json | 61 -
.../resources/apispec/cluster.Commands.json | 25 -
.../apispec/collections.Commands.json | 11 -
...ollections.collection.Commands.modify.json | 11 -
.../autoscaling/testAddMissingReplica.json | 122 -
.../autoscaling/testAddTooManyPerPolicy.json | 129 -
.../testAutoScalingHandlerFailure.json | 141 -
...utoscalingPreferencesUsedWithNoPolicy.json | 52 -
.../testComputePlanAfterNodeAdded.json | 16 -
.../autoscaling/testCoresSuggestions.json | 17 -
.../testCreateCollectionWithEmptyPolicy.json | 20 -
.../solr/autoscaling/testDiskSpaceHint.json | 16 -
.../solr/autoscaling/testEmptyCollection.json | 26 -
.../solr/autoscaling/testEqualOnNonNode.json | 83 -
.../autoscaling/testFreeDiskDeviation.json | 35 -
.../autoscaling/testFreeDiskSuggestions.json | 27 -
.../autoscaling/testFreediskPercentage.json | 25 -
.../solr/autoscaling/testHostAttribute.json | 119 -
.../solr/autoscaling/testInfiniteLoop.json | 13173 ----------------
.../autoscaling/testMoveReplicaSuggester.json | 15 -
...testMoveReplicasInMultipleCollections.json | 86 -
.../solrj/solr/autoscaling/testPolicy.json | 41 -
.../solr/autoscaling/testPortSuggestions.json | 22 -
.../testReplicaCountSuggestions.json | 15 -
.../autoscaling/testReplicaPercentage.json | 46 -
.../testReplicaZonesPercentage.json | 15 -
.../testScheduledTriggerFailure.json | 52 -
.../solrj/solr/autoscaling/testSortError.json | 225 -
.../testSuggestionsRebalance2.json | 131 -
.../testSuggestionsRebalanceOnly.json | 105 -
.../autoscaling/testSysPropSuggestions.json | 127 -
.../autoscaling/testSyspropSuggestions1.json | 24 -
.../autoscaling/testUnresolvedSuggestion.json | 211 -
.../autoscaling/testUtilizeNodeFailure.json | 69 -
.../autoscaling/testUtilizeNodeFailure2.json | 66 -
.../solr/autoscaling/testViolationOutput.json | 22 -
.../solr/autoscaling/testWithCollection.json | 21 -
.../testWithCollectionMoveReplica.json | 28 -
...estWithCollectionMoveVsAddSuggestions.json | 49 -
.../testWithCollectionSuggestions.json | 21 -
.../cloud/autoscaling/ConditionTest.java | 80 -
.../autoscaling/MoveReplicaSuggesterTest.java | 104 -
.../solrj/cloud/autoscaling/TestPolicy.java | 3358 ----
.../solrj/cloud/autoscaling/TestPolicy2.java | 540 -
.../cloud/autoscaling/TestPolicy2Old.java | 26 -
.../cloud/autoscaling/TestPolicyOld.java | 25 -
.../solrj/impl/CloudSolrClientCacheTest.java | 3 +-
solr/webapp/web/css/angular/menu.css | 1 -
solr/webapp/web/css/angular/suggestions.css | 64 -
solr/webapp/web/index.html | 3 -
solr/webapp/web/js/angular/app.js | 4 -
.../web/js/angular/controllers/cloud.js | 2 -
.../controllers/cluster-suggestions.js | 62 -
.../web/js/angular/controllers/collections.js | 6 +-
.../web/partials/cluster_suggestions.html | 49 -
.../web/partials/collection_overview.html | 3 -
solr/webapp/web/partials/collections.html | 12 -
287 files changed, 606 insertions(+), 80223 deletions(-)
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/api/collections/UtilizeNodeCmd.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/ActionContext.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveMarkersPlanAction.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/LoggingListener.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionException.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/package-info.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/ActionError.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/NoopDistributedQueueFactory.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotCloudManager.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotClusterStateProvider.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotDistribStateManager.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotNodeStateProvider.java
delete mode 100644 solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/package-info.java
delete mode 100644 solr/core/src/java/org/apache/solr/handler/admin/AutoscalingHistoryHandler.java
delete mode 100644 solr/core/src/test-files/solr/simSnapshot/autoscalingState.json
delete mode 100644 solr/core/src/test-files/solr/simSnapshot/clusterState.json
delete mode 100644 solr/core/src/test-files/solr/simSnapshot/distribState.json
delete mode 100644 solr/core/src/test-files/solr/simSnapshot/managerState.json
delete mode 100644 solr/core/src/test-files/solr/simSnapshot/nodeState.json
delete mode 100644 solr/core/src/test-files/solr/simSnapshot/statistics.json
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/ReplaceNodeNoTargetTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/RoutingToNodesWithPropertiesTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/TestUtilizeNode.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/TestWithCollection.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/api/collections/AssignTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/api/collections/ConcurrentCreateCollectionTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/HdfsAutoAddReplicasIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerMixedBoundsTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerSizeEstimationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/RestoreTriggerStateTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledTriggerIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledTriggerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/SystemLogListenerTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerCooldownIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerEventQueueTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerSetPropertiesIntegrationTest.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimClusterStateProvider.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimComputePlanAction.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistribStateManager.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExecutePlanAction.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimExtremeIndexing.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimGenericDistributedQueue.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeAddedTrigger.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimNodeLostTrigger.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimScenario.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimUtils.java
delete mode 100644 solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSnapshotCloudManager.java
delete mode 100644 solr/core/src/test/org/apache/solr/handler/admin/AutoscalingHistoryHandlerTest.java
delete mode 100644 solr/solr-ref-guide/src/colocating-collections.adoc
delete mode 100644 solr/solr-ref-guide/src/migrate-to-policy-rule.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-auto-add-replicas.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-fault-tolerance.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-listeners.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-trigger-actions.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
delete mode 100644 solr/solr-ref-guide/src/solrcloud-autoscaling.adoc
delete mode 100644 solr/solr-ref-guide/src/suggestions-screen.adoc
rename solr/solrj/src/java/org/apache/solr/client/solrj/cloud/{autoscaling => }/AlreadyExistsException.java (94%)
rename solr/solrj/src/java/org/apache/solr/client/solrj/cloud/{autoscaling => }/BadVersionException.java (95%)
create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/DelegatingCloudManager.java
create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/DelegatingClusterStateProvider.java
rename solr/solrj/src/java/org/apache/solr/client/solrj/cloud/{autoscaling => }/NotEmptyException.java (94%)
rename solr/solrj/src/java/org/apache/solr/client/solrj/cloud/{autoscaling => }/VersionedData.java (97%)
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AddReplicaSuggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Cell.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ComputedType.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Condition.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingClusterStateProvider.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DeleteNodeSuggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DeleteReplicaSuggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/MoveReplicaSuggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeVariable.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NoneSuggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Operand.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Preference.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/RangeVal.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaVariable.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Row.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SealedClause.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SplitShardSuggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventProcessorStage.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/UnsupportedSuggester.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/WithCollectionVariable.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/package-info.java
delete mode 100644 solr/solrj/src/java/org/apache/solr/common/params/AutoScalingParams.java
delete mode 100644 solr/solrj/src/resources/apispec/autoscaling.Commands.json
delete mode 100644 solr/solrj/src/resources/apispec/autoscaling.history.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testAddMissingReplica.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testAddTooManyPerPolicy.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testAutoScalingHandlerFailure.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testAutoscalingPreferencesUsedWithNoPolicy.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testComputePlanAfterNodeAdded.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testCoresSuggestions.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testCreateCollectionWithEmptyPolicy.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testDiskSpaceHint.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testEmptyCollection.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testEqualOnNonNode.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testFreeDiskDeviation.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testFreeDiskSuggestions.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testFreediskPercentage.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testHostAttribute.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testInfiniteLoop.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testMoveReplicaSuggester.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testMoveReplicasInMultipleCollections.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testPolicy.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testPortSuggestions.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testReplicaCountSuggestions.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testReplicaPercentage.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testReplicaZonesPercentage.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testScheduledTriggerFailure.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testSortError.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testSuggestionsRebalance2.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testSuggestionsRebalanceOnly.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testSysPropSuggestions.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testSyspropSuggestions1.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testUnresolvedSuggestion.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testUtilizeNodeFailure.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testUtilizeNodeFailure2.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testViolationOutput.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testWithCollection.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testWithCollectionMoveReplica.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testWithCollectionMoveVsAddSuggestions.json
delete mode 100644 solr/solrj/src/test-files/solrj/solr/autoscaling/testWithCollectionSuggestions.json
delete mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/ConditionTest.java
delete mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/MoveReplicaSuggesterTest.java
delete mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
delete mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
delete mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2Old.java
delete mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicyOld.java
delete mode 100644 solr/webapp/web/css/angular/suggestions.css
delete mode 100644 solr/webapp/web/js/angular/controllers/cluster-suggestions.js
delete mode 100644 solr/webapp/web/partials/cluster_suggestions.html
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 53ca9e0fa2e..ee5afa17669 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -36,6 +36,8 @@ Improvements
Other Changes
----------------------
+* SOLR-14656: Autoscaling framework removed (Ishan Chattopadhyaya, noble, Ilan Ginzburg)
+
* LUCENE-9391: Upgrade HPPC to 0.8.2. (Haoyu Zhai)
* SOLR-10288: Remove non-minified JavaScript from the webapp. (Erik Hatcher, marcussorealheis)
diff --git a/solr/bin/solr b/solr/bin/solr
index 83d247ccefa..7e3cf0c62d6 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -330,7 +330,7 @@ function print_usage() {
if [ -z "$CMD" ]; then
echo ""
echo "Usage: solr COMMAND OPTIONS"
- echo " where COMMAND is one of: start, stop, restart, status, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, autoscaling, export"
+ echo " where COMMAND is one of: start, stop, restart, status, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, export"
echo ""
echo " Standalone server example (start Solr running in the background on port 8984):"
echo ""
@@ -1417,11 +1417,6 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
fi
-if [[ "$SCRIPT_CMD" == "autoscaling" ]]; then
- run_tool autoscaling $@
- exit $?
-fi
-
if [[ "$SCRIPT_CMD" == "export" ]]; then
run_tool export $@
exit $?
diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd
index 4b4d8132c5b..8fb5e7ad4e5 100755
--- a/solr/bin/solr.cmd
+++ b/solr/bin/solr.cmd
@@ -224,7 +224,6 @@ IF "%1"=="version" goto get_version
IF "%1"=="-v" goto get_version
IF "%1"=="-version" goto get_version
IF "%1"=="assert" goto run_assert
-IF "%1"=="autoscaling" goto run_autoscaling
IF "%1"=="export" goto run_export
IF "%1"=="package" goto run_package
@@ -303,7 +302,7 @@ goto done
:script_usage
@echo.
@echo Usage: solr COMMAND OPTIONS
-@echo where COMMAND is one of: start, stop, restart, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, autoscaling, export
+@echo where COMMAND is one of: start, stop, restart, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, export
@echo.
@echo Standalone server example (start Solr running in the background on port 8984):
@echo.
@@ -1454,13 +1453,6 @@ if errorlevel 1 (
)
goto done
-:run_autoscaling
-"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
- -Dlog4j.configurationFile="file:///%DEFAULT_SERVER_DIR%\resources\log4j2-console.xml" ^
- -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^
- org.apache.solr.util.SolrCLI %*
-goto done:
-
:run_export
"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
-Dlog4j.configurationFile="file:///%DEFAULT_SERVER_DIR%\resources\log4j2-console.xml" ^
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 ecc653b52f0..65de4ca6804 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
@@ -235,7 +235,6 @@ public class CloudUtil {
* Return a {@link CollectionStatePredicate} that returns true if a collection has the expected
* number of shards and replicas.
* Note: for shards marked as inactive the current Solr behavior is that replicas remain active.
- * {@link org.apache.solr.cloud.autoscaling.sim.SimCloudManager} follows this behavior.
* @param expectedShards expected number of shards
* @param expectedReplicas expected number of active replicas per shard
* @param withInactive if true then count also inactive shards
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 3dfe6c593e9..bb405ad4863 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -39,7 +39,6 @@ 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.autoscaling.OverseerTriggerThread;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.cloud.overseer.CollectionMutator;
import org.apache.solr.cloud.overseer.NodeMutator;
@@ -83,16 +82,14 @@ import com.codahale.metrics.Timer;
* collections, shards, replicas and setting various properties.
*
* The Overseer is a single elected node in the SolrCloud cluster that is in charge of interactions with
- * ZooKeeper that require global synchronization. It also hosts the Collection API implementation and the
- * Autoscaling framework.
+ * ZooKeeper that require global synchronization.
*
* The Overseer deals with:
*
* Cluster State updates, i.e. updating Collections' state.json
files in ZooKeeper, see {@link ClusterStateUpdater},
- * Collection API implementation, including Autoscaling replica placement computation, see
+ * Collection API implementation, see
* {@link OverseerCollectionConfigSetProcessor} and {@link OverseerCollectionMessageHandler} (and the example below),
* Updating Config Sets, see {@link OverseerCollectionConfigSetProcessor} and {@link OverseerConfigSetMessageHandler},
- * Autoscaling triggers, see {@link org.apache.solr.cloud.autoscaling.OverseerTriggerThread}.
*
*
* The nodes in the cluster communicate with the Overseer over queues implemented in ZooKeeper. There are essentially
@@ -644,14 +641,8 @@ public class Overseer implements SolrCloseable {
ccThread = new OverseerThread(ccTg, overseerCollectionConfigSetProcessor, "OverseerCollectionConfigSetProcessor-" + id);
ccThread.setDaemon(true);
- ThreadGroup triggerThreadGroup = new ThreadGroup("Overseer autoscaling triggers");
- OverseerTriggerThread trigger = new OverseerTriggerThread(zkController.getCoreContainer().getResourceLoader(),
- zkController.getSolrCloudManager());
- triggerThread = new OverseerThread(triggerThreadGroup, trigger, "OverseerAutoScalingTriggerThread-" + id);
-
updaterThread.start();
ccThread.start();
- triggerThread.start();
systemCollectionCompatCheck(new BiConsumer() {
boolean firstPair = true;
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 c50581cc539..ca44ece55cf 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -60,7 +60,6 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.AlreadyClosedException;
@@ -99,7 +98,6 @@ import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.URLUtil;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloseHook;
@@ -897,13 +895,8 @@ public class ZkController implements Closeable {
cmdExecutor.ensureExists(ZkStateReader.LIVE_NODES_ZKNODE, zkClient);
cmdExecutor.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE, zkClient);
cmdExecutor.ensureExists(ZkStateReader.ALIASES, zkClient);
- cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH, zkClient);
- cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, zkClient);
- cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, zkClient);
- cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, zkClient);
byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
cmdExecutor.ensureExists(ZkStateReader.SOLR_SECURITY_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
- cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
bootstrapDefaultConfigSet(zkClient);
}
@@ -1044,29 +1037,6 @@ public class ZkController implements Closeable {
}
i++;
}
-
- // retrieve current trigger config - if there are no nodeLost triggers
- // then don't create markers
- boolean createNodes = false;
- try {
- createNodes = zkStateReader.getAutoScalingConfig().hasTriggerForEvents(TriggerEventType.NODELOST);
- } catch (KeeperException | InterruptedException e1) {
- log.warn("Unable to read autoscaling.json", e1);
- }
- if (createNodes) {
- byte[] json = Utils.toJSON(Collections.singletonMap("timestamp", getSolrCloudManager().getTimeSource().getEpochTimeNs()));
- for (String n : oldNodes) {
- String path = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + n;
-
- try {
- zkClient.create(path, json, CreateMode.PERSISTENT, true);
- } catch (KeeperException.NodeExistsException e) {
- // someone else already created this node - ignore
- } catch (KeeperException | InterruptedException e1) {
- log.warn("Unable to register nodeLost path for {}", n, e1);
- }
- }
- }
return false;
};
zkStateReader.registerLiveNodesListener(listener);
@@ -1152,18 +1122,9 @@ public class ZkController implements Closeable {
}
String nodeName = getNodeName();
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
- String nodeAddedPath = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName;
log.info("Register node as live in ZooKeeper:{}", nodePath);
List ops = new ArrayList<>(2);
ops.add(Op.create(nodePath, null, zkClient.getZkACLProvider().getACLsToAdd(nodePath), CreateMode.EPHEMERAL));
- // if there are nodeAdded triggers don't create nodeAdded markers
- boolean createMarkerNode = zkStateReader.getAutoScalingConfig().hasTriggerForEvents(TriggerEventType.NODEADDED);
- if (createMarkerNode && !zkClient.exists(nodeAddedPath, true)) {
- // use EPHEMERAL so that it disappears if this node goes down
- // and no other action is taken
- byte[] json = Utils.toJSON(Collections.singletonMap("timestamp", TimeSource.NANO_TIME.getEpochTimeNs()));
- ops.add(Op.create(nodeAddedPath, json, zkClient.getZkACLProvider().getACLsToAdd(nodeAddedPath), CreateMode.EPHEMERAL));
- }
zkClient.multi(ops, true);
}
@@ -1173,11 +1134,9 @@ public class ZkController implements Closeable {
}
String nodeName = getNodeName();
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
- String nodeAddedPath = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName;
log.info("Remove node as live in ZooKeeper:{}", nodePath);
List ops = new ArrayList<>(2);
ops.add(Op.delete(nodePath, -1));
- ops.add(Op.delete(nodeAddedPath, -1));
try {
zkClient.multi(ops, true);
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 1c2146bbfb3..ca5639f4989 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
@@ -44,12 +44,10 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
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.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.ActiveReplicaWatcher;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
@@ -148,18 +146,11 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
- AtomicReference sessionWrapper = new AtomicReference<>();
- List createReplicas;
- try {
- createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount, sessionWrapper)
+ List createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
.stream()
.map(replicaPosition -> assignReplicaDetails(ocmh.cloudManager, clusterState, message, replicaPosition))
.collect(Collectors.toList());
- } finally {
- if (sessionWrapper.get() != null) {
- sessionWrapper.get().release();
- }
- }
+
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ZkStateReader zkStateReader = ocmh.zkStateReader;
@@ -339,8 +330,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
public static List buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
String collectionName, ZkNodeProps message,
- EnumMap replicaTypeVsCount,
- AtomicReference< PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+ EnumMap replicaTypeVsCount) throws IOException, InterruptedException {
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
boolean skipNodeAssignment = message.getBool(CollectionAdminParams.SKIP_NODE_ASSIGNMENT, false);
String sliceName = message.getStr(SHARD_ID_PROP);
@@ -365,7 +355,6 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
positions = Assign.getNodesForNewReplicas(clusterState, collection.getName(), sliceName, numNrtReplicas,
numTlogReplicas, numPullReplicas, createNodeSetStr, cloudManager);
- sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
}
if (positions == null) {
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 98a399a79d5..909b3eddc99 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
@@ -16,6 +16,10 @@
*/
package org.apache.solr.cloud.api.collections;
+import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
+import static org.apache.solr.common.cloud.DocCollection.SNITCH;
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
@@ -29,19 +33,15 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
-import java.util.Optional;
import java.util.Random;
import java.util.Set;
import java.util.stream.Collectors;
-import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.BadVersionException;
+import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.cloud.rule.ReplicaAssigner;
import org.apache.solr.cloud.rule.Rule;
import org.apache.solr.common.SolrException;
@@ -52,19 +52,14 @@ import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.cloud.Slice;
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.util.StrUtils;
-import org.apache.solr.common.util.Utils;
import org.apache.solr.util.NumberUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.POLICY;
-import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
-import static org.apache.solr.common.cloud.DocCollection.SNITCH;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import com.google.common.collect.ImmutableMap;
public class Assign {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -255,57 +250,6 @@ public class Assign {
return nodeList;
}
- /**
- * Note: where possible, the {@link #usePolicyFramework(DocCollection, SolrCloudManager)} method should
- * be used instead of this method
- *
- * @return true if autoscaling policy framework should be used for replica placement
- */
- public static boolean usePolicyFramework(SolrCloudManager cloudManager) throws IOException, InterruptedException {
- Objects.requireNonNull(cloudManager, "The SolrCloudManager instance cannot be null");
- return usePolicyFramework(Optional.empty(), cloudManager);
- }
-
- /**
- * @return true if auto scaling policy framework should be used for replica placement
- * for this collection, otherwise false
- */
- public static boolean usePolicyFramework(DocCollection collection, SolrCloudManager cloudManager)
- throws IOException, InterruptedException {
- Objects.requireNonNull(collection, "The DocCollection instance cannot be null");
- Objects.requireNonNull(cloudManager, "The SolrCloudManager instance cannot be null");
- return usePolicyFramework(Optional.of(collection), cloudManager);
- }
-
- @SuppressWarnings({"unchecked"})
- private static boolean usePolicyFramework(Optional collection, SolrCloudManager cloudManager) throws IOException, InterruptedException {
- boolean useLegacyAssignment = true;
- Map clusterProperties = cloudManager.getClusterStateProvider().getClusterProperties();
- if (clusterProperties.containsKey(CollectionAdminParams.DEFAULTS)) {
- Map defaults = (Map) clusterProperties.get(CollectionAdminParams.DEFAULTS);
- Map collectionDefaults = (Map) defaults.getOrDefault(CollectionAdminParams.CLUSTER, Collections.emptyMap());
- useLegacyAssignment = Boolean.parseBoolean(collectionDefaults.getOrDefault(CollectionAdminParams.USE_LEGACY_REPLICA_ASSIGNMENT, "true").toString());
- }
-
- if (!useLegacyAssignment) {
- // if legacy assignment is not selected then autoscaling is always available through the implicit policy/preferences
- return true;
- }
-
- // legacy assignment is turned on, which means we must look at the actual autoscaling config
- // to determine whether policy framework can be used or not for this collection
-
- AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
- // if no autoscaling configuration exists then obviously we cannot use the policy framework
- if (autoScalingConfig.getPolicy().isEmpty()) return false;
- // do custom preferences exist
- if (!autoScalingConfig.getPolicy().hasEmptyPreferences()) return true;
- // does a cluster policy exist
- if (!autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) return true;
- // finally we check if the current collection has a policy
- return !collection.isPresent() || collection.get().getPolicyName() != null;
- }
-
static class ReplicaCount {
public final String nodeName;
public int thisCollectionNodes = 0;
@@ -358,44 +302,6 @@ public class Assign {
return assignStrategy.assign(cloudManager, assignRequest);
}
- public static List getPositionsUsingPolicy(String collName, List shardNames,
- int nrtReplicas,
- int tlogReplicas,
- int pullReplicas,
- String policyName, SolrCloudManager cloudManager,
- List nodesList) throws IOException, InterruptedException, AssignmentException {
- log.debug("shardnames {} NRT {} TLOG {} PULL {} , policy {}, nodeList {}", shardNames, nrtReplicas, tlogReplicas, pullReplicas, policyName, nodesList);
- List replicaPositions = null;
- AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
- try {
- Map kvMap = Collections.singletonMap(collName, policyName);
- replicaPositions = PolicyHelper.getReplicaLocations(
- collName,
- autoScalingConfig,
- cloudManager,
- kvMap,
- shardNames,
- nrtReplicas,
- tlogReplicas,
- pullReplicas,
- nodesList);
- return replicaPositions;
- } catch (Exception e) {
- throw new AssignmentException("Error getting replica locations : " + e.getMessage(), e);
- } finally {
- if (log.isTraceEnabled()) {
- if (replicaPositions != null) {
- if (log.isTraceEnabled()) {
- log.trace("REPLICA_POSITIONS: {}", Utils.toJSONString(Utils.getDeepCopy(replicaPositions, 7, true)));
- }
- }
- if (log.isTraceEnabled()) {
- log.trace("AUTOSCALING_CONF: {}", Utils.toJSONString(autoScalingConfig));
- }
- }
- }
- }
-
static HashMap getNodeNameVsShardCount(String collectionName,
ClusterState clusterState, List createNodeList) {
HashMap nodeNameVsShardCount = new HashMap<>();
@@ -637,22 +543,6 @@ public class Assign {
}
}
- public static class PolicyBasedAssignStrategy implements AssignStrategy {
- public String policyName;
-
- public PolicyBasedAssignStrategy(String policyName) {
- this.policyName = policyName;
- }
-
- @Override
- public List assign(SolrCloudManager solrCloudManager, AssignRequest assignRequest) throws Assign.AssignmentException, IOException, InterruptedException {
- return Assign.getPositionsUsingPolicy(assignRequest.collectionName,
- assignRequest.shardNames, assignRequest.numNrtReplicas,
- assignRequest.numTlogReplicas, assignRequest.numPullReplicas,
- policyName, solrCloudManager, assignRequest.nodes);
- }
- }
-
public static class AssignStrategyFactory {
public SolrCloudManager solrCloudManager;
@@ -663,19 +553,16 @@ public class Assign {
public AssignStrategy create(ClusterState clusterState, DocCollection collection) throws IOException, InterruptedException {
@SuppressWarnings({"unchecked", "rawtypes"})
List ruleMaps = (List) collection.get("rule");
- String policyName = collection.getStr(POLICY);
@SuppressWarnings({"rawtypes"})
List snitches = (List) collection.get(SNITCH);
Strategy strategy = null;
- if ((ruleMaps == null || ruleMaps.isEmpty()) && !usePolicyFramework(collection, solrCloudManager)) {
- strategy = Strategy.LEGACY;
- } else if (ruleMaps != null && !ruleMaps.isEmpty()) {
+ if (ruleMaps != null && !ruleMaps.isEmpty()) {
strategy = Strategy.RULES;
} else {
- strategy = Strategy.POLICY;
+ strategy = Strategy.LEGACY;
}
-
+
switch (strategy) {
case LEGACY:
return new LegacyAssignStrategy();
@@ -683,15 +570,13 @@ public class Assign {
List rules = new ArrayList<>();
for (Object map : ruleMaps) rules.add(new Rule((Map) map));
return new RulesBasedAssignStrategy(rules, snitches, clusterState);
- case POLICY:
- return new PolicyBasedAssignStrategy(policyName);
default:
throw new Assign.AssignmentException("Unknown strategy type: " + strategy);
}
}
private enum Strategy {
- LEGACY, RULES, POLICY;
+ LEGACY, RULES;
}
}
}
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 c3e9a3e64fb..cfad397fbd5 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
@@ -31,15 +31,13 @@ import java.util.NoSuchElementException;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
-import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
-import org.apache.solr.client.solrj.cloud.autoscaling.NotEmptyException;
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.BadVersionException;
+import org.apache.solr.client.solrj.cloud.NotEmptyException;
+import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
@@ -147,7 +145,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
List shardNames = populateShardNames(message, router);
checkReplicaTypes(message);
- AtomicReference sessionWrapper = new AtomicReference<>();
+
try {
@@ -187,7 +185,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
List replicaPositions = null;
try {
- replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, clusterState.getCollection(collectionName), message, shardNames, sessionWrapper);
+ replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, clusterState.getCollection(collectionName), message, shardNames);
} catch (Assign.AssignmentException e) {
ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
@@ -335,16 +333,13 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
throw ex;
} catch (Exception ex) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
- } finally {
- if (sessionWrapper.get() != null) sessionWrapper.get().release();
}
}
public static List buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
DocCollection docCollection,
ZkNodeProps message,
- List shardNames,
- AtomicReference sessionWrapper) throws IOException, InterruptedException, Assign.AssignmentException {
+ List shardNames) throws IOException, InterruptedException, Assign.AssignmentException {
final String collectionName = message.getStr(NAME);
// look at the replication factor and see if it matches reality
// if it does not, find best nodes to create more cores
@@ -386,7 +381,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(cloudManager);
Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, docCollection);
replicaPositions = assignStrategy.assign(cloudManager, assignRequest);
- sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
}
return replicaPositions;
}
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 e1fca930be4..74112b48acf 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
@@ -38,8 +38,8 @@ import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
-import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.BadVersionException;
import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
@@ -91,7 +91,6 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.POLICY;
import static org.apache.solr.common.cloud.DocCollection.SNITCH;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
@@ -146,9 +145,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
ZkStateReader.NRT_REPLICAS, "1",
ZkStateReader.TLOG_REPLICAS, "0",
ZkStateReader.PULL_REPLICAS, "0",
- ZkStateReader.AUTO_ADD_REPLICAS, "false",
DocCollection.RULE, null,
- POLICY, null,
SNITCH, null,
WITH_COLLECTION, null,
COLOCATED_WITH, null));
@@ -238,7 +235,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
.put(ADDREPLICA, new AddReplicaCmd(this))
.put(MOVEREPLICA, new MoveReplicaCmd(this))
.put(REINDEXCOLLECTION, new ReindexCollectionCmd(this))
- .put(UTILIZENODE, new UtilizeNodeCmd(this))
.put(RENAME, new RenameCmd(this))
.build()
;
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 d98d50a3f4d..c45c772e518 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
@@ -35,7 +35,6 @@ import com.google.common.annotations.VisibleForTesting;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@@ -110,10 +109,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
ZkStateReader.TLOG_REPLICAS,
ZkStateReader.REPLICATION_FACTOR,
"shards",
- Policy.POLICY,
CollectionAdminParams.CREATE_NODE_SET_PARAM,
- CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM,
- ZkStateReader.AUTO_ADD_REPLICAS
+ CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM
);
private final OverseerCollectionMessageHandler ocmh;
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 f1c1f8cc8a3..aa10bb1e4b7 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,9 +27,6 @@ import java.util.Locale;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.ActiveReplicaWatcher;
import org.apache.solr.common.SolrCloseableLatch;
import org.apache.solr.common.SolrException;
@@ -101,7 +98,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(sourceReplicas.size(), ocmh);
SolrCloseableLatch replicasToRecover = new SolrCloseableLatch(numLeaders, ocmh);
- AtomicReference sessionWrapperRef = new AtomicReference<>();
try {
for (ZkNodeProps sourceReplica : sourceReplicas) {
@SuppressWarnings({"rawtypes"})
@@ -127,7 +123,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
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;
- sessionWrapperRef.set(PolicyHelper.getLastSessionWrapper(true));
}
ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
if (async != null) msg.getProperties().put(ASYNC, async);
@@ -185,8 +180,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
log.debug("Finished waiting for replicas to be added");
}
} finally {
- PolicyHelper.SessionWrapper sw = sessionWrapperRef.get();
- if (sw != null) sw.release();
}
// now wait for leader replicas to recover
log.debug("Waiting for {} leader replicas to recover", numLeaders);
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 f80097913dc..552c40a0dea 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
@@ -35,7 +35,6 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.overseer.OverseerAction;
@@ -110,15 +109,15 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
Object format = properties.get("stateFormat");
if (format != null && !"2".equals(format)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Collection " + backupCollection + " is in stateFormat=" + format +
- " no longer supported in Solr 9 and above. It can't be restored. If it originates in Solr 8 you can restore" +
- " it there, migrate it to stateFormat=2 and backup again, it will then be restorable on Solr 9");
+ " no longer supported in Solr 9 and above. It can't be restored. If it originates in Solr 8 you can restore" +
+ " it there, migrate it to stateFormat=2 and backup again, it will then be restorable on Solr 9");
}
String backupCollectionAlias = properties.getProperty(BackupManager.COLLECTION_ALIAS_PROP);
DocCollection backupCollectionState = backupMgr.readCollectionState(location, backupName, backupCollection);
// Get the Solr nodes to restore a collection.
final List nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(
- zkStateReader.getClusterState().getLiveNodes(), message, OverseerCollectionMessageHandler.RANDOM);
+ zkStateReader.getClusterState().getLiveNodes(), message, OverseerCollectionMessageHandler.RANDOM);
int numShards = backupCollectionState.getActiveSlices().size();
@@ -136,7 +135,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
int numPullReplicas = getInt(message, PULL_REPLICAS, backupCollectionState.getNumPullReplicas(), 0);
int totalReplicasPerShard = numNrtReplicas + numTlogReplicas + numPullReplicas;
assert totalReplicasPerShard > 0;
-
+
//Upload the configs
String configName = (String) properties.get(CollectionAdminParams.COLL_CONF);
String restoreConfigName = message.getStr(CollectionAdminParams.COLL_CONF, configName);
@@ -149,7 +148,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
log.info("Starting restore into collection={} with backup_name={} at location={}", restoreCollectionName, backupName,
- location);
+ location);
//Create core-less collection
{
@@ -190,7 +189,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
Map newSlices = new LinkedHashMap<>(backupSlices.size());
for (Slice backupSlice : backupSlices) {
newSlices.put(backupSlice.getName(),
- new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(),restoreCollectionName));
+ new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(), restoreCollectionName));
}
propMap.put(OverseerCollectionMessageHandler.SHARDS_PROP, newSlices);
}
@@ -222,216 +221,211 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
List sliceNames = new ArrayList<>();
restoreCollection.getSlices().forEach(x -> sliceNames.add(x.getName()));
- PolicyHelper.SessionWrapper sessionWrapper = null;
- try {
- Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
- .forCollection(restoreCollectionName)
- .forShard(sliceNames)
- .assignNrtReplicas(numNrtReplicas)
- .assignTlogReplicas(numTlogReplicas)
- .assignPullReplicas(numPullReplicas)
- .onNodes(nodeList)
- .build();
- Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
- Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, restoreCollection);
- List replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
- sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
+ Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
+ .forCollection(restoreCollectionName)
+ .forShard(sliceNames)
+ .assignNrtReplicas(numNrtReplicas)
+ .assignTlogReplicas(numTlogReplicas)
+ .assignPullReplicas(numPullReplicas)
+ .onNodes(nodeList)
+ .build();
+ Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
+ Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, restoreCollection);
+ List replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
- CountDownLatch countDownLatch = new CountDownLatch(restoreCollection.getSlices().size());
+ CountDownLatch countDownLatch = new CountDownLatch(restoreCollection.getSlices().size());
- //Create one replica per shard and copy backed up data to it
- for (Slice slice : restoreCollection.getSlices()) {
- if (log.isInfoEnabled()) {
- log.info("Adding replica for shard={} collection={} ", slice.getName(), restoreCollection);
+ //Create one replica per shard and copy backed up data to it
+ for (Slice slice : restoreCollection.getSlices()) {
+ if (log.isInfoEnabled()) {
+ log.info("Adding replica for shard={} collection={} ", slice.getName(), restoreCollection);
+ }
+ HashMap propMap = new HashMap<>();
+ propMap.put(Overseer.QUEUE_OPERATION, CREATESHARD);
+ propMap.put(COLLECTION_PROP, restoreCollectionName);
+ propMap.put(SHARD_ID_PROP, slice.getName());
+
+ if (numNrtReplicas >= 1) {
+ propMap.put(REPLICA_TYPE, Replica.Type.NRT.name());
+ } else if (numTlogReplicas >= 1) {
+ propMap.put(REPLICA_TYPE, Replica.Type.TLOG.name());
+ } else {
+ throw new SolrException(ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " +
+ Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
+ }
+
+ // Get the first node matching the shard to restore in
+ String node;
+ for (ReplicaPosition replicaPosition : replicaPositions) {
+ if (Objects.equals(replicaPosition.shard, slice.getName())) {
+ node = replicaPosition.node;
+ propMap.put(CoreAdminParams.NODE, node);
+ replicaPositions.remove(replicaPosition);
+ break;
}
- HashMap propMap = new HashMap<>();
- propMap.put(Overseer.QUEUE_OPERATION, CREATESHARD);
- propMap.put(COLLECTION_PROP, restoreCollectionName);
- propMap.put(SHARD_ID_PROP, slice.getName());
+ }
- if (numNrtReplicas >= 1) {
- propMap.put(REPLICA_TYPE, Replica.Type.NRT.name());
- } else if (numTlogReplicas >= 1) {
- propMap.put(REPLICA_TYPE, Replica.Type.TLOG.name());
+ // add async param
+ if (asyncId != null) {
+ propMap.put(ASYNC, asyncId);
+ }
+ 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 {
- throw new SolrException(ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " +
- Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
- }
-
- // Get the first node matching the shard to restore in
- String node;
- for (ReplicaPosition replicaPosition : replicaPositions) {
- if (Objects.equals(replicaPosition.shard, slice.getName())) {
- node = replicaPosition.node;
- propMap.put(CoreAdminParams.NODE, node);
- replicaPositions.remove(replicaPosition);
- break;
+ SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
+ if (success == null) {
+ success = new SimpleOrderedMap();
+ results.add("success", success);
}
+ success.addAll((NamedList) addReplicaResult.get("success"));
}
-
- // add async param
- if (asyncId != null) {
- propMap.put(ASYNC, asyncId);
- }
- 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();
- });
- }
-
- boolean allIsDone = countDownLatch.await(1, TimeUnit.HOURS);
- if (!allIsDone) {
- throw new TimeoutException("Initial replicas were not created within 1 hour. Timing out.");
- }
- Object failures = results.get("failure");
- if (failures != null && ((SimpleOrderedMap) failures).size() > 0) {
- log.error("Restore failed to create initial replicas.");
- ocmh.cleanupCollection(restoreCollectionName, new NamedList());
- return;
- }
-
- //refresh the location copy of collection state
- restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
-
- {
- ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
- // Copy data from backed up index to each replica
- for (Slice slice : restoreCollection.getSlices()) {
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.RESTORECORE.toString());
- params.set(NAME, "snapshot." + slice.getName());
- params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString());
- params.set(CoreAdminParams.BACKUP_REPOSITORY, repo);
- shardRequestTracker.sliceCmd(clusterState, params, null, slice, shardHandler);
- }
- shardRequestTracker.processResponses(new NamedList(), shardHandler, true, "Could not restore core");
- }
-
- {
- ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
-
- for (Slice s : restoreCollection.getSlices()) {
- for (Replica r : s.getReplicas()) {
- String nodeName = r.getNodeName();
- String coreNodeName = r.getCoreName();
- Replica.State stateRep = r.getState();
-
- if (log.isDebugEnabled()) {
- log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}", nodeName, coreNodeName,
- stateRep.name());
- }
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
- params.set(CoreAdminParams.NAME, coreNodeName);
-
- shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
- }
-
- shardRequestTracker.processResponses(new NamedList(), shardHandler, true,
- "REQUESTAPPLYUPDATES calls did not succeed");
- }
- }
-
- //Mark all shards in ACTIVE STATE
- {
- HashMap propMap = new HashMap<>();
- propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
- propMap.put(ZkStateReader.COLLECTION_PROP, restoreCollectionName);
- for (Slice shard : restoreCollection.getSlices()) {
- propMap.put(shard.getName(), Slice.State.ACTIVE.toString());
- }
- ocmh.overseer.offerStateUpdate((Utils.toJSON(new ZkNodeProps(propMap))));
- }
-
- if (totalReplicasPerShard > 1) {
- if (log.isInfoEnabled()) {
- log.info("Adding replicas to restored collection={}", restoreCollection.getName());
- }
- for (Slice slice : restoreCollection.getSlices()) {
-
- //Add the remaining replicas for each shard, considering it's type
- int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
-
- // We already created either a NRT or an TLOG replica as leader
- if (numNrtReplicas > 0) {
- createdNrtReplicas++;
- } else if (createdTlogReplicas > 0) {
- createdTlogReplicas++;
- }
-
- for (int i = 1; i < totalReplicasPerShard; i++) {
- Replica.Type typeToCreate;
- if (createdNrtReplicas < numNrtReplicas) {
- createdNrtReplicas++;
- typeToCreate = Replica.Type.NRT;
- } else if (createdTlogReplicas < numTlogReplicas) {
- createdTlogReplicas++;
- typeToCreate = Replica.Type.TLOG;
- } else {
- createdPullReplicas++;
- typeToCreate = Replica.Type.PULL;
- assert createdPullReplicas <= numPullReplicas: "Unexpected number of replicas";
- }
-
- if (log.isDebugEnabled()) {
- log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
- }
- HashMap propMap = new HashMap<>();
- propMap.put(COLLECTION_PROP, restoreCollectionName);
- propMap.put(SHARD_ID_PROP, slice.getName());
- propMap.put(REPLICA_TYPE, typeToCreate.name());
-
- // Get the first node matching the shard to restore in
- String node;
- for (ReplicaPosition replicaPosition : replicaPositions) {
- if (Objects.equals(replicaPosition.shard, slice.getName())) {
- node = replicaPosition.node;
- propMap.put(CoreAdminParams.NODE, node);
- replicaPositions.remove(replicaPosition);
- break;
- }
- }
-
- // add async param
- if (asyncId != null) {
- propMap.put(ASYNC, asyncId);
- }
- ocmh.addPropertyParams(message, propMap);
-
- ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
- }
- }
- }
-
- if (backupCollectionAlias != null && !backupCollectionAlias.equals(backupCollection)) {
- log.debug("Restoring alias {} -> {}", backupCollectionAlias, backupCollection);
- ocmh.zkStateReader.aliasesManager
- .applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(backupCollectionAlias, backupCollection));
- }
-
- log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
- } finally {
- if (sessionWrapper != null) sessionWrapper.release();
+ countDownLatch.countDown();
+ });
}
+
+ boolean allIsDone = countDownLatch.await(1, TimeUnit.HOURS);
+ if (!allIsDone) {
+ throw new TimeoutException("Initial replicas were not created within 1 hour. Timing out.");
+ }
+ Object failures = results.get("failure");
+ if (failures != null && ((SimpleOrderedMap) failures).size() > 0) {
+ log.error("Restore failed to create initial replicas.");
+ ocmh.cleanupCollection(restoreCollectionName, new NamedList());
+ return;
+ }
+
+ //refresh the location copy of collection state
+ restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
+
+ {
+ ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
+ // Copy data from backed up index to each replica
+ for (Slice slice : restoreCollection.getSlices()) {
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.RESTORECORE.toString());
+ params.set(NAME, "snapshot." + slice.getName());
+ params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString());
+ params.set(CoreAdminParams.BACKUP_REPOSITORY, repo);
+ shardRequestTracker.sliceCmd(clusterState, params, null, slice, shardHandler);
+ }
+ shardRequestTracker.processResponses(new NamedList(), shardHandler, true, "Could not restore core");
+ }
+
+ {
+ ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
+
+ for (Slice s : restoreCollection.getSlices()) {
+ for (Replica r : s.getReplicas()) {
+ String nodeName = r.getNodeName();
+ String coreNodeName = r.getCoreName();
+ Replica.State stateRep = r.getState();
+
+ if (log.isDebugEnabled()) {
+ log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}", nodeName, coreNodeName,
+ stateRep.name());
+ }
+
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
+ params.set(CoreAdminParams.NAME, coreNodeName);
+
+ shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
+ }
+
+ shardRequestTracker.processResponses(new NamedList(), shardHandler, true,
+ "REQUESTAPPLYUPDATES calls did not succeed");
+ }
+ }
+
+ //Mark all shards in ACTIVE STATE
+ {
+ HashMap propMap = new HashMap<>();
+ propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
+ propMap.put(ZkStateReader.COLLECTION_PROP, restoreCollectionName);
+ for (Slice shard : restoreCollection.getSlices()) {
+ propMap.put(shard.getName(), Slice.State.ACTIVE.toString());
+ }
+ ocmh.overseer.offerStateUpdate((Utils.toJSON(new ZkNodeProps(propMap))));
+ }
+
+ if (totalReplicasPerShard > 1) {
+ if (log.isInfoEnabled()) {
+ log.info("Adding replicas to restored collection={}", restoreCollection.getName());
+ }
+ for (Slice slice : restoreCollection.getSlices()) {
+
+ //Add the remaining replicas for each shard, considering it's type
+ int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
+
+ // We already created either a NRT or an TLOG replica as leader
+ if (numNrtReplicas > 0) {
+ createdNrtReplicas++;
+ } else if (createdTlogReplicas > 0) {
+ createdTlogReplicas++;
+ }
+
+ for (int i = 1; i < totalReplicasPerShard; i++) {
+ Replica.Type typeToCreate;
+ if (createdNrtReplicas < numNrtReplicas) {
+ createdNrtReplicas++;
+ typeToCreate = Replica.Type.NRT;
+ } else if (createdTlogReplicas < numTlogReplicas) {
+ createdTlogReplicas++;
+ typeToCreate = Replica.Type.TLOG;
+ } else {
+ createdPullReplicas++;
+ typeToCreate = Replica.Type.PULL;
+ assert createdPullReplicas <= numPullReplicas : "Unexpected number of replicas";
+ }
+
+ if (log.isDebugEnabled()) {
+ log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
+ }
+ HashMap propMap = new HashMap<>();
+ propMap.put(COLLECTION_PROP, restoreCollectionName);
+ propMap.put(SHARD_ID_PROP, slice.getName());
+ propMap.put(REPLICA_TYPE, typeToCreate.name());
+
+ // Get the first node matching the shard to restore in
+ String node;
+ for (ReplicaPosition replicaPosition : replicaPositions) {
+ if (Objects.equals(replicaPosition.shard, slice.getName())) {
+ node = replicaPosition.node;
+ propMap.put(CoreAdminParams.NODE, node);
+ replicaPositions.remove(replicaPosition);
+ break;
+ }
+ }
+
+ // add async param
+ if (asyncId != null) {
+ propMap.put(ASYNC, asyncId);
+ }
+ ocmh.addPropertyParams(message, propMap);
+
+ ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
+ }
+ }
+ }
+
+ if (backupCollectionAlias != null && !backupCollectionAlias.equals(backupCollection)) {
+ log.debug("Restoring alias {} -> {}", backupCollectionAlias, backupCollection);
+ ocmh.zkStateReader.aliasesManager
+ .applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(backupCollectionAlias, backupCollection));
+ }
+
+ log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
+
}
private int getInt(ZkNodeProps message, String propertyName, Integer count, int defaultValue) {
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 0df9d09f03b..495bf65985e 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
@@ -18,41 +18,17 @@
package org.apache.solr.cloud.api.collections;
-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.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+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.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.CompositeIdRouter;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.DocRouter;
-import org.apache.solr.common.cloud.PlainIdRouter;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.ReplicaPosition;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.*;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.params.CommonAdminParams;
import org.apache.solr.common.params.CommonParams;
@@ -71,13 +47,15 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
-import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import java.lang.invoke.MethodHandles;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.solr.client.solrj.impl.SolrClientNodeStateProvider.Variable.CORE_IDX;
+import static org.apache.solr.common.cloud.ZkStateReader.*;
import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
@@ -135,7 +113,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
String splitKey = message.getStr("split.key");
DocCollection collection = clusterState.getCollection(collectionName);
- PolicyHelper.SessionWrapper sessionWrapper = null;
Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
if (parentSlice.getState() != Slice.State.ACTIVE) {
@@ -427,7 +404,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
log.debug("Successfully applied buffered updates on : {}", subShardNames);
// Replica creation for the new Slices
- // replica placement is controlled by the autoscaling policy framework
Set nodes = clusterState.getLiveNodes();
List nodeList = new ArrayList<>(nodes.size());
@@ -458,7 +434,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, collection);
List replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
- sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
t.stop();
t = timings.sub("createReplicaPlaceholders");
@@ -611,7 +586,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
log.error("Error executing split operation for collection: {} parent shard: {}", collectionName, slice, e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
} finally {
- if (sessionWrapper != null) sessionWrapper.release();
if (!success) {
cleanupAfterFailure(zkStateReader, collectionName, parentSlice.getName(), subSlices, offlineSlices);
unlockForSplit(ocmh.cloudManager, collectionName, parentSlice.getName());
@@ -638,7 +612,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
Map nodeValues = nodeStateProvider.getNodeValues(parentShardLeader.getNodeName(),
Collections.singletonList(ImplicitSnitch.DISK));
Map>> infos = nodeStateProvider.getReplicaInfo(parentShardLeader.getNodeName(),
- Collections.singletonList(Type.CORE_IDX.metricsAttribute));
+ Collections.singletonList(CORE_IDX.metricsAttribute));
if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing replica information for parent shard leader");
}
@@ -647,11 +621,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
Double indexSize = null;
for (Replica info : lst) {
if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
- Number size = (Number)info.get(Type.CORE_IDX.metricsAttribute);
+ Number size = (Number)info.get( CORE_IDX.metricsAttribute);
if (size == null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing index size information for parent shard leader");
}
- indexSize = (Double) Type.CORE_IDX.convertVal(size);
+ indexSize = (Double) CORE_IDX.convertVal(size);
break;
}
}
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/UtilizeNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/UtilizeNodeCmd.java
deleted file mode 100644
index 0fd19ac21db..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/UtilizeNodeCmd.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.api.collections;
-
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
-import org.apache.solr.client.solrj.request.V2Request;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
-import static org.apache.solr.common.params.AutoScalingParams.NODE;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
-import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
-
-public class UtilizeNodeCmd implements OverseerCollectionMessageHandler.Cmd {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- private final OverseerCollectionMessageHandler ocmh;
-
- public UtilizeNodeCmd(OverseerCollectionMessageHandler ocmh) {
- this.ocmh = ocmh;
- }
-
- @Override
- public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
- ocmh.checkRequired(message, NODE);
- String nodeName = message.getStr(NODE);
- String async = message.getStr(ASYNC);
- AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
-
- //first look for any violation that may use this replica
- List requests = new ArrayList<>();
- //first look for suggestions if any
- List suggestions = PolicyHelper.getSuggestions(autoScalingConfig, ocmh.overseer.getSolrCloudManager());
- for (Suggester.SuggestionInfo suggestionInfo : suggestions) {
- if (log.isInfoEnabled()) {
- log.info("op: {}", suggestionInfo.getOperation());
- }
- String coll = null;
- List pieces = StrUtils.splitSmart(suggestionInfo.getOperation().getPath(), '/');
- if (pieces.size() > 1) {
- coll = pieces.get(2);
- } else {
- continue;
- }
- log.info("coll: {}", coll);
- if (suggestionInfo.getOperation() instanceof V2Request) {
- String targetNode = (String) Utils.getObjectByPath(suggestionInfo.getOperation(), true, "command/move-replica/targetNode");
- if (Objects.equals(targetNode, nodeName)) {
- String replica = (String) Utils.getObjectByPath(suggestionInfo.getOperation(), true, "command/move-replica/replica");
- requests.add(new ZkNodeProps(COLLECTION_PROP, coll,
- CollectionParams.TARGET_NODE, targetNode,
- ASYNC, async,
- REPLICA_PROP, replica));
- }
- }
- }
- executeAll(requests);
- PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getSession(ocmh.overseer.getSolrCloudManager());
- Policy.Session session = sessionWrapper.get();
- Suggester initialsuggester = session.getSuggester(MOVEREPLICA)
- .hint(Suggester.Hint.TARGET_NODE, nodeName);
- Suggester suggester = null;
- for (; ; ) {
- suggester = session.getSuggester(MOVEREPLICA)
- .hint(Suggester.Hint.TARGET_NODE, nodeName);
- @SuppressWarnings({"rawtypes"})
- SolrRequest request = suggester.getSuggestion();
- if (requests.size() > 10) {
- log.info("too_many_suggestions");
- PolicyHelper.logState(ocmh.overseer.getSolrCloudManager(), initialsuggester);
- break;
- }
- log.info("SUGGESTION: {}", request);
- if (request == null) break;
- session = suggester.getSession();
- requests.add(new ZkNodeProps(COLLECTION_PROP, request.getParams().get(COLLECTION_PROP),
- CollectionParams.TARGET_NODE, request.getParams().get(CollectionParams.TARGET_NODE),
- REPLICA_PROP, request.getParams().get(REPLICA_PROP),
- ASYNC, request.getParams().get(ASYNC)));
- }
- if (log.isInfoEnabled()) {
- log.info("total_suggestions: {}", requests.size());
- }
- if (requests.size() == 0) {
- PolicyHelper.logState(ocmh.overseer.getSolrCloudManager(), initialsuggester);
- }
- sessionWrapper.returnSession(session);
- try {
- executeAll(requests);
- } finally {
- sessionWrapper.release();
- }
- }
-
- private void executeAll(List requests) throws Exception {
- if (requests.isEmpty()) return;
- for (ZkNodeProps props : requests) {
- @SuppressWarnings({"rawtypes"})
- NamedList result = new NamedList();
- ocmh.commandMap.get(MOVEREPLICA)
- .call(ocmh.overseer.getSolrCloudManager().getClusterStateProvider().getClusterState(),
- props,
- result);
- }
- requests.clear();
- }
-
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ActionContext.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ActionContext.java
deleted file mode 100644
index 8487d3d7cf2..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ActionContext.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.common.MapWriter;
-
-/**
- * Provides additional context for the TriggerAction such as the trigger instance on
- * which the action is being executed as well as helper methods to pass computed information along
- * to the next action
- */
-public class ActionContext implements MapWriter {
-
- private final SolrCloudManager cloudManager;
- private final AutoScaling.Trigger source;
- private final Map properties;
-
- public ActionContext(SolrCloudManager cloudManager, AutoScaling.Trigger source, Map properties) {
- this.cloudManager = cloudManager;
- this.source = source;
- this.properties = properties;
- }
-
- public SolrCloudManager getCloudManager() {
- return cloudManager;
- }
-
- public AutoScaling.Trigger getSource() {
- return source;
- }
-
- public Map getProperties() {
- return properties;
- }
-
- public Object getProperty(String name) {
- return properties != null ? properties.get(name) : null;
- }
-
- @Override
- public void writeMap(EntryWriter ew) throws IOException {
- ew.put("source", source.getName());
- if (properties != null) {
- for (Map.Entry entry : properties.entrySet()) {
- ew.put("properties." + entry.getKey(), entry.getValue());
- }
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java
deleted file mode 100644
index d129fdbefe9..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.core.SolrResourceLoader;
-
-import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
-
-/**
- * This class configures the parent ComputePlanAction to compute plan
- * only for collections which have autoAddReplicas=true.
- */
-public class AutoAddReplicasPlanAction extends ComputePlanAction {
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- properties.put("collections", Collections.singletonMap(AUTO_ADD_REPLICAS, "true"));
- super.configure(loader, cloudManager, properties);
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
deleted file mode 100644
index 1a191ee858b..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-
-public class AutoScaling {
-
- /**
- * Implementation of this interface is used for processing events generated by a trigger.
- */
- public interface TriggerEventProcessor {
-
- /**
- * This method is executed for events produced by {@link Trigger#run()}.
- *
- * @param event a subclass of {@link TriggerEvent}
- * @return true if the processor was ready to perform actions on the event, false
- * otherwise. If false was returned then callers should assume the event was discarded.
- */
- boolean process(TriggerEvent event);
- }
-
- /**
- * Interface for a Solr trigger. Each trigger implements Runnable and Closeable interface. A trigger
- * is scheduled using a {@link java.util.concurrent.ScheduledExecutorService} so it is executed as
- * per a configured schedule to check whether the trigger is ready to fire. The {@link AutoScaling.Trigger#setProcessor(AutoScaling.TriggerEventProcessor)}
- * method should be used to set a processor which is used by implementation of this class whenever
- * ready.
- *
- * As per the guarantees made by the {@link java.util.concurrent.ScheduledExecutorService} a trigger
- * implementation is only ever called sequentially and therefore need not be thread safe. However, it
- * is encouraged that implementations be immutable with the exception of the associated listener
- * which can be get/set by a different thread than the one executing the trigger. Therefore, implementations
- * should use appropriate synchronization around the listener.
- *
- * When a trigger is ready to fire, it calls the {@link TriggerEventProcessor#process(TriggerEvent)} event
- * with the proper trigger event object. If that method returns false then it should be interpreted to mean
- * that Solr is not ready to process this trigger event and therefore we should retain the state and fire
- * at the next invocation of the run() method.
- */
- public interface Trigger extends Closeable, Runnable {
- /**
- * Trigger name.
- */
- String getName();
-
- /**
- * Event type generated by this trigger.
- */
- TriggerEventType getEventType();
-
- /** Returns true if this trigger is enabled. */
- boolean isEnabled();
-
- /** Trigger properties. */
- Map getProperties();
-
- /** Number of seconds to wait between fired events ("waitFor" property). */
- int getWaitForSecond();
-
- /** Actions to execute when event is fired. */
- List getActions();
-
- /** Set event processor to call when event is fired. */
- void setProcessor(TriggerEventProcessor processor);
-
- /** Get event processor. */
- TriggerEventProcessor getProcessor();
-
- /** Return true when this trigger is closed and cannot be used. */
- boolean isClosed();
-
- /** Set internal state of this trigger from another instance. */
- void restoreState(Trigger old);
-
- /** Save internal state of this trigger in ZooKeeper. */
- void saveState();
-
- /** Restore internal state of this trigger from ZooKeeper. */
- void restoreState();
-
- /**
- * Called when trigger is created but before it's initialized or scheduled for use.
- * This method should also verify that the trigger configuration parameters are correct. It may
- * be called multiple times.
- * @param properties configuration properties
- * @throws TriggerValidationException contains details of invalid configuration parameters.
- */
- void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException;
-
- /**
- * Called before a trigger is scheduled. Any heavy object creation or initialisation should
- * be done in this method instead of the Trigger's constructor.
- */
- void init() throws Exception;
- }
-
- /**
- * Factory to produce instances of {@link Trigger}.
- */
- public static abstract class TriggerFactory implements Closeable {
- protected boolean isClosed = false;
-
- public abstract Trigger create(TriggerEventType type, String name, Map props) throws TriggerValidationException;
-
- @Override
- public void close() throws IOException {
- synchronized (this) {
- isClosed = true;
- }
- }
- }
-
- /**
- * Default implementation of {@link TriggerFactory}.
- */
- public static class TriggerFactoryImpl extends TriggerFactory {
-
- private final SolrCloudManager cloudManager;
- private final SolrResourceLoader loader;
-
- public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager cloudManager) {
- Objects.requireNonNull(cloudManager);
- Objects.requireNonNull(loader);
- this.cloudManager = cloudManager;
- this.loader = loader;
- }
-
- @Override
- public synchronized Trigger create(TriggerEventType type, String name, Map props) throws TriggerValidationException {
- if (isClosed) {
- throw new AlreadyClosedException("TriggerFactory has already been closed, cannot create new triggers");
- }
- if (type == null) {
- throw new IllegalArgumentException("Trigger type must not be null");
- }
- if (name == null || name.isEmpty()) {
- throw new IllegalArgumentException("Trigger name must not be empty");
- }
- Trigger t;
- switch (type) {
- case NODEADDED:
- t = new NodeAddedTrigger(name);
- break;
- case NODELOST:
- t = new NodeLostTrigger(name);
- break;
- case SEARCHRATE:
- t = new SearchRateTrigger(name);
- break;
- case METRIC:
- t = new MetricTrigger(name);
- break;
- case SCHEDULED:
- t = new ScheduledTrigger(name);
- break;
- case INDEXSIZE:
- t = new IndexSizeTrigger(name);
- break;
- default:
- throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
- }
- t.configure(loader, cloudManager, props);
- return t;
- }
-
- }
-
- public static final String AUTO_ADD_REPLICAS_TRIGGER_NAME = ".auto_add_replicas";
-
- public static final String AUTO_ADD_REPLICAS_TRIGGER_DSL =
- " {" +
- " 'name' : '" + AUTO_ADD_REPLICAS_TRIGGER_NAME + "'," +
- " 'event' : 'nodeLost'," +
- " 'waitFor' : -1," +
- " 'enabled' : true," +
- " 'actions' : [" +
- " {" +
- " 'name':'auto_add_replicas_plan'," +
- " 'class':'solr.AutoAddReplicasPlanAction'" +
- " }," +
- " {" +
- " 'name':'execute_plan'," +
- " 'class':'solr.ExecutePlanAction'" +
- " }" +
- " ]" +
- " }";
-
- @SuppressWarnings({"unchecked"})
- public static final Map AUTO_ADD_REPLICAS_TRIGGER_PROPS = (Map) Utils.fromJSONString(AUTO_ADD_REPLICAS_TRIGGER_DSL);
-
- public static final String SCHEDULED_MAINTENANCE_TRIGGER_NAME = ".scheduled_maintenance";
-
- public static final String SCHEDULED_MAINTENANCE_TRIGGER_DSL =
- " {" +
- " 'name' : '" + SCHEDULED_MAINTENANCE_TRIGGER_NAME + "'," +
- " 'event' : 'scheduled'," +
- " 'startTime' : 'NOW'," +
- " 'every' : '+1DAY'," +
- " 'enabled' : true," +
- " 'actions' : [" +
- " {" +
- " 'name':'inactive_shard_plan'," +
- " 'class':'solr.InactiveShardPlanAction'" +
- " }," +
- " {" +
- " 'name':'inactive_markers_plan'," +
- " 'class':'solr.InactiveMarkersPlanAction'" +
- " }," +
- " {" +
- " 'name':'execute_plan'," +
- " 'class':'solr.ExecutePlanAction'" +
- " }" +
- " ]" +
- " }";
-
- @SuppressWarnings({"unchecked"})
- public static final Map SCHEDULED_MAINTENANCE_TRIGGER_PROPS = (Map) Utils.fromJSONString(SCHEDULED_MAINTENANCE_TRIGGER_DSL);
-
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
deleted file mode 100644
index 23ec0754cbc..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
+++ /dev/null
@@ -1,742 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiConsumer;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import org.apache.solr.api.Api;
-import org.apache.solr.api.ApiBag;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
-import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
-import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.common.MapWriter;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.AutoScalingParams;
-import org.apache.solr.common.params.CollectionAdminParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.CommandOperation;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.handler.RequestHandlerBase;
-import org.apache.solr.handler.RequestHandlerUtils;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.security.AuthorizationContext;
-import org.apache.solr.security.PermissionNameProvider;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static java.util.stream.Collectors.collectingAndThen;
-import static java.util.stream.Collectors.toSet;
-import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
-import static org.apache.solr.common.params.AutoScalingParams.*;
-import static org.apache.solr.common.params.CommonParams.JSON;
-
-/**
- * Handler for /cluster/autoscaling
- */
-public class AutoScalingHandler extends RequestHandlerBase implements PermissionNameProvider {
- public static final String HANDLER_PATH = "/admin/autoscaling";
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- protected final SolrCloudManager cloudManager;
- protected final SolrResourceLoader loader;
- protected final AutoScaling.TriggerFactory triggerFactory;
- private final List> DEFAULT_ACTIONS = new ArrayList<>(3);
- private static Set singletonCommands = Stream.of("set-cluster-preferences", "set-cluster-policy")
- .collect(collectingAndThen(toSet(), Collections::unmodifiableSet));
-
- private final TimeSource timeSource;
-
- public AutoScalingHandler(SolrCloudManager cloudManager, SolrResourceLoader loader) {
- this.cloudManager = cloudManager;
- this.loader = loader;
- this.triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
- this.timeSource = cloudManager.getTimeSource();
- Map map = new HashMap<>(2);
- map.put(NAME, "compute_plan");
- map.put(CLASS, "solr.ComputePlanAction");
- DEFAULT_ACTIONS.add(map);
- map = new HashMap<>(2);
- map.put(NAME, "execute_plan");
- map.put(CLASS, "solr.ExecutePlanAction");
- DEFAULT_ACTIONS.add(map);
- }
-
- Optional> getSubpathExecutor(List path, SolrQueryRequest request) {
- if (path.size() == 3) {
- if (DIAGNOSTICS.equals(path.get(2))) {
- return Optional.of((rsp, autoScalingConf) -> handleDiagnostics(rsp, autoScalingConf));
- } else if (SUGGESTIONS.equals(path.get(2))) {
- return Optional.of((rsp, autoScalingConf) -> handleSuggestions(rsp, autoScalingConf, request.getParams()));
- } else {
- return Optional.empty();
- }
-
- }
- return Optional.empty();
- }
-
- @Override
- @SuppressWarnings({"unchecked", "rawtypes"})
- public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
- try {
- String httpMethod = (String) req.getContext().get("httpMethod");
- RequestHandlerUtils.setWt(req, JSON);
-
- if ("GET".equals(httpMethod)) {
- String path = (String) req.getContext().get("path");
- if (path == null) path = "/cluster/autoscaling";
- List parts = StrUtils.splitSmart(path, '/', true);
-
- if (parts.size() < 2 || parts.size() > 3) {
- // invalid
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
- }
-
- AutoScalingConfig autoScalingConf = cloudManager.getDistribStateManager().getAutoScalingConfig();
- if (parts.size() == 2) {
- autoScalingConf.writeMap(new MapWriter.EntryWriter() {
-
- @Override
- public MapWriter.EntryWriter put(CharSequence k, Object v) {
- rsp.getValues().add(k.toString(), v);
- return this;
- }
- });
- } else {
- getSubpathExecutor(parts, req).ifPresent(it -> it.accept(rsp, autoScalingConf));
- }
- } else {
- if (req.getContentStreams() == null) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No commands specified for autoscaling");
- }
- String path = (String) req.getContext().get("path");
- if (path != null) {
- List parts = StrUtils.splitSmart(path, '/', true);
- if(parts.size() == 3){
- getSubpathExecutor(parts, req).ifPresent(it -> {
- Map map = null;
- try {
- map = (Map) Utils.fromJSON(req.getContentStreams().iterator().next().getStream());
- } catch (IOException e1) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "error parsing payload", e1);
- }
- it.accept(rsp, new AutoScalingConfig(map));
- });
-
- return;
- }
-
- }
- List ops = CommandOperation.readCommands(req.getContentStreams(), rsp.getValues(), singletonCommands);
- if (ops == null) {
- // errors have already been added to the response so there's nothing left to do
- return;
- }
- processOps(req, rsp, ops);
- }
-
- } catch (Exception e) {
- rsp.getValues().add("result", "failure");
- throw e;
- } finally {
- RequestHandlerUtils.addExperimentalFormatWarning(rsp);
- }
- }
-
-
- @SuppressWarnings({"unchecked"})
- private void handleSuggestions(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf, SolrParams params) {
- rsp.getValues().add("suggestions",
- PolicyHelper.getSuggestions(autoScalingConf, cloudManager, params));
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- public void processOps(SolrQueryRequest req, SolrQueryResponse rsp, List ops)
- throws KeeperException, InterruptedException, IOException {
- while (true) {
- AutoScalingConfig initialConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
- AutoScalingConfig currentConfig = initialConfig;
- for (CommandOperation op : ops) {
- switch (op.name) {
- case CMD_SET_TRIGGER:
- currentConfig = handleSetTrigger(req, rsp, op, currentConfig);
- break;
- case CMD_REMOVE_TRIGGER:
- currentConfig = handleRemoveTrigger(req, rsp, op, currentConfig);
- break;
- case CMD_SET_LISTENER:
- currentConfig = handleSetListener(req, rsp, op, currentConfig);
- break;
- case CMD_REMOVE_LISTENER:
- currentConfig = handleRemoveListener(req, rsp, op, currentConfig);
- break;
- case CMD_SUSPEND_TRIGGER:
- currentConfig = handleSuspendTrigger(req, rsp, op, currentConfig);
- break;
- case CMD_RESUME_TRIGGER:
- currentConfig = handleResumeTrigger(req, rsp, op, currentConfig);
- break;
- case CMD_SET_POLICY:
- currentConfig = handleSetPolicies(req, rsp, op, currentConfig);
- break;
- case CMD_REMOVE_POLICY:
- currentConfig = handleRemovePolicy(req, rsp, op, currentConfig);
- break;
- case CMD_SET_CLUSTER_PREFERENCES:
- currentConfig = handleSetClusterPreferences(req, rsp, op, currentConfig);
- break;
- case CMD_SET_CLUSTER_POLICY:
- currentConfig = handleSetClusterPolicy(req, rsp, op, currentConfig);
- break;
- case CMD_SET_PROPERTIES:
- currentConfig = handleSetProperties(req, rsp, op, currentConfig);
- break;
- default:
- op.addError("Unknown command: " + op.name);
- }
- }
- List errs = CommandOperation.captureErrors(ops);
- if (!errs.isEmpty()) {
- throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "Error in command payload", errs);
- }
-
- if (!currentConfig.equals(initialConfig)) {
- // update in ZK
- if (setAutoScalingConfig(currentConfig)) {
- break;
- } else {
- // someone else updated the config, get the latest one and re-apply our ops
- rsp.getValues().add("retry", "initialVersion=" + initialConfig.getZkVersion());
- continue;
- }
- } else {
- // no changes
- break;
- }
- }
- rsp.getValues().add("result", "success");
- }
-
- private AutoScalingConfig handleSetProperties(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op, AutoScalingConfig currentConfig) {
- Map map = op.getDataMap() == null ? Collections.emptyMap() : op.getDataMap();
- Map configProps = new HashMap<>(currentConfig.getProperties());
- configProps.putAll(map);
- // remove a key which is set to null
- map.forEach((k, v) -> {
- if (v == null) configProps.remove(k);
- });
- return currentConfig.withProperties(configProps);
- }
-
- @SuppressWarnings({"unchecked"})
- private void handleDiagnostics(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) {
- Policy policy = autoScalingConf.getPolicy();
- rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, cloudManager));
- }
-
- @SuppressWarnings({"unchecked"})
- private AutoScalingConfig handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
- List> clusterPolicy = (List>) op.getCommandData();
- if (clusterPolicy == null || !(clusterPolicy instanceof List)) {
- op.addError("set-cluster-policy expects an array of objects");
- return currentConfig;
- }
- List cp = null;
- try {
- cp = clusterPolicy.stream().map(Clause::create).collect(Collectors.toList());
- } catch (Exception e) {
- op.addError(e.getMessage());
- return currentConfig;
- }
- Policy p = currentConfig.getPolicy().withClusterPolicy(cp);
- currentConfig = currentConfig.withPolicy(p);
- return currentConfig;
- }
-
- @SuppressWarnings({"unchecked"})
- private AutoScalingConfig handleSetClusterPreferences(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
- List> preferences = (List>) op.getCommandData();
- if (preferences == null || !(preferences instanceof List)) {
- op.addError("A list of cluster preferences not found");
- return currentConfig;
- }
- List prefs = null;
- try {
- prefs = preferences.stream().map(Preference::new).collect(Collectors.toList());
- } catch (Exception e) {
- op.addError(e.getMessage());
- return currentConfig;
- }
- Policy p = currentConfig.getPolicy().withClusterPreferences(prefs);
- currentConfig = currentConfig.withPolicy(p);
- return currentConfig;
- }
-
- private AutoScalingConfig handleRemovePolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
- String policyName = (String) op.getVal("");
-
- if (op.hasError()) return currentConfig;
-
- Map> policies = currentConfig.getPolicy().getPolicies();
- if (policies == null || !policies.containsKey(policyName)) {
- op.addError("No policy exists with name: " + policyName);
- return currentConfig;
- }
-
- cloudManager.getClusterStateProvider().getClusterState().forEachCollection(coll -> {
- if (policyName.equals(coll.getPolicyName()))
- op.addError(StrUtils.formatString("policy : {0} is being used by collection {1}", policyName, coll.getName()));
- });
- if (op.hasError()) return currentConfig;
- policies = new HashMap<>(policies);
- policies.remove(policyName);
- Policy p = currentConfig.getPolicy().withPolicies(policies);
- currentConfig = currentConfig.withPolicy(p);
- return currentConfig;
- }
-
- @SuppressWarnings({"unchecked"})
- private AutoScalingConfig handleSetPolicies(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
- Map policiesMap = op.getDataMap();
- for (Map.Entry policy : policiesMap.entrySet()) {
- String policyName = policy.getKey();
- if (policyName == null || policyName.trim().length() == 0) {
- op.addError("The policy name cannot be null or empty");
- return currentConfig;
- }
- }
- Map> currentClauses = new HashMap<>(currentConfig.getPolicy().getPolicies());
- Map> newClauses = null;
- try {
- newClauses = Policy.clausesFromMap((Map>>) op.getCommandData(),
- new ArrayList<>() );
- } catch (Exception e) {
- op.addError(e.getMessage());
- return currentConfig;
- }
- currentClauses.putAll(newClauses);
- Policy p = currentConfig.getPolicy().withPolicies(currentClauses);
- currentConfig = currentConfig.withPolicy(p);
- return currentConfig;
- }
-
- @SuppressWarnings({"unchecked"})
- private AutoScalingConfig handleResumeTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
- String triggerName = op.getStr(NAME);
- if (op.hasError()) return currentConfig;
- Map triggers = currentConfig.getTriggerConfigs();
- Set changed = new HashSet<>();
- if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
- op.addError("No trigger exists with name: " + triggerName);
- return currentConfig;
- }
- Map newTriggers = new HashMap<>();
- for (Map.Entry entry : triggers.entrySet()) {
- if (Policy.EACH.equals(triggerName) || triggerName.equals(entry.getKey())) {
- AutoScalingConfig.TriggerConfig trigger = entry.getValue();
- if (!trigger.enabled) {
- trigger = trigger.withEnabled(true);
- newTriggers.put(entry.getKey(), trigger);
- changed.add(entry.getKey());
- } else {
- newTriggers.put(entry.getKey(), entry.getValue());
- }
- } else {
- newTriggers.put(entry.getKey(), entry.getValue());
- }
- }
- rsp.getValues().add("changed", changed);
- if (!changed.isEmpty()) {
- currentConfig = currentConfig.withTriggerConfigs(newTriggers);
- }
- return currentConfig;
- }
-
- @SuppressWarnings({"unchecked"})
- private AutoScalingConfig handleSuspendTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
- String triggerName = op.getStr(NAME);
- if (op.hasError()) return currentConfig;
- String timeout = op.getStr(TIMEOUT, null);
- Date resumeTime = null;
- if (timeout != null) {
- try {
- int timeoutSeconds = parseHumanTime(timeout);
- resumeTime = new Date(TimeUnit.MILLISECONDS.convert(timeSource.getTimeNs(), TimeUnit.NANOSECONDS)
- + TimeUnit.MILLISECONDS.convert(timeoutSeconds, TimeUnit.SECONDS));
- } catch (IllegalArgumentException e) {
- op.addError("Invalid 'timeout' value for suspend trigger: " + triggerName);
- return currentConfig;
- }
- }
-
- Map triggers = currentConfig.getTriggerConfigs();
- Set changed = new HashSet<>();
-
- if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
- op.addError("No trigger exists with name: " + triggerName);
- return currentConfig;
- }
- Map newTriggers = new HashMap<>();
- for (Map.Entry entry : triggers.entrySet()) {
- if (Policy.EACH.equals(triggerName) || triggerName.equals(entry.getKey())) {
- AutoScalingConfig.TriggerConfig trigger = entry.getValue();
- if (trigger.enabled) {
- trigger = trigger.withEnabled(false);
- if (resumeTime != null) {
- trigger = trigger.withProperty(RESUME_AT, resumeTime.getTime());
- }
- newTriggers.put(entry.getKey(), trigger);
- changed.add(trigger.name);
- } else {
- newTriggers.put(entry.getKey(), entry.getValue());
- }
- } else {
- newTriggers.put(entry.getKey(), entry.getValue());
- }
- }
- rsp.getValues().add("changed", changed);
- if (!changed.isEmpty()) {
- currentConfig = currentConfig.withTriggerConfigs(newTriggers);
- }
- return currentConfig;
- }
-
- private AutoScalingConfig handleRemoveListener(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
- String listenerName = op.getStr(NAME);
-
- if (op.hasError()) return currentConfig;
- Map listeners = currentConfig.getTriggerListenerConfigs();
- if (listeners == null || !listeners.containsKey(listenerName)) {
- op.addError("No listener exists with name: " + listenerName);
- return currentConfig;
- }
- currentConfig = currentConfig.withoutTriggerListenerConfig(listenerName);
- return currentConfig;
- }
-
- private AutoScalingConfig handleSetListener(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
- String listenerName = op.getStr(NAME);
- String triggerName = op.getStr(TRIGGER);
- List stageNames = op.getStrs(STAGE, Collections.emptyList());
- String listenerClass = op.getStr(CLASS);
- List beforeActions = op.getStrs(BEFORE_ACTION, Collections.emptyList());
- List afterActions = op.getStrs(AFTER_ACTION, Collections.emptyList());
-
- if (op.hasError()) return currentConfig;
-
- Map triggers = currentConfig.getTriggerConfigs();
- if (triggers == null || !triggers.containsKey(triggerName)) {
- op.addError("A trigger with the name " + triggerName + " does not exist");
- return currentConfig;
- }
- AutoScalingConfig.TriggerConfig triggerConfig = triggers.get(triggerName);
-
- if (stageNames.isEmpty() && beforeActions.isEmpty() && afterActions.isEmpty()) {
- op.addError("Either 'stage' or 'beforeAction' or 'afterAction' must be specified");
- return currentConfig;
- }
-
- for (String stage : stageNames) {
- try {
- TriggerEventProcessorStage.valueOf(stage);
- } catch (IllegalArgumentException e) {
- op.addError("Invalid stage name: " + stage);
- }
- }
- if (op.hasError()) return currentConfig;
-
- AutoScalingConfig.TriggerListenerConfig listenerConfig = new AutoScalingConfig.TriggerListenerConfig(listenerName, op.getValuesExcluding("name"));
-
- // validate that we can load the listener class
- // todo allow creation from blobstore
- TriggerListener listener = null;
- try {
- listener = loader.newInstance(listenerClass, TriggerListener.class);
- listener.configure(loader, cloudManager, listenerConfig);
- } catch (TriggerValidationException e) {
- log.warn("invalid listener configuration", e);
- op.addError("invalid listener configuration: " + e.toString());
- return currentConfig;
- } catch (Exception e) {
- log.warn("error loading listener class ", e);
- op.addError("Listener not found: " + listenerClass + ". error message:" + e.getMessage());
- return currentConfig;
- } finally {
- if (listener != null) {
- IOUtils.closeQuietly(listener);
- }
- }
-
- Set actionNames = new HashSet<>();
- actionNames.addAll(beforeActions);
- actionNames.addAll(afterActions);
- for (AutoScalingConfig.ActionConfig action : triggerConfig.actions) {
- actionNames.remove(action.name);
- }
- if (!actionNames.isEmpty()) {
- op.addError("The trigger '" + triggerName + "' does not have actions named: " + actionNames);
- return currentConfig;
- }
- // todo - handle races between competing set-trigger and set-listener invocations
- currentConfig = currentConfig.withTriggerListenerConfig(listenerConfig);
- return currentConfig;
- }
-
- @SuppressWarnings({"unchecked"})
- private AutoScalingConfig handleSetTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
- // we're going to modify the op - use a copy
- String triggerName = op.getStr(NAME);
- String eventTypeStr = op.getStr(EVENT);
-
- if (op.hasError()) return currentConfig;
- TriggerEventType eventType = TriggerEventType.valueOf(eventTypeStr.trim().toUpperCase(Locale.ROOT));
-
- String waitForStr = op.getStr(WAIT_FOR, null);
-
- CommandOperation opCopy = new CommandOperation(op.name, Utils.getDeepCopy((Map) op.getCommandData(), 10));
-
- if (waitForStr != null) {
- int seconds = 0;
- try {
- seconds = parseHumanTime(waitForStr);
- } catch (IllegalArgumentException e) {
- op.addError("Invalid 'waitFor' value '" + waitForStr + "' in trigger: " + triggerName);
- return currentConfig;
- }
- opCopy.getDataMap().put(WAIT_FOR, seconds);
- }
-
- Integer lowerBound = op.getInt(LOWER_BOUND, null);
- Integer upperBound = op.getInt(UPPER_BOUND, null);
-
- List> actions = (List>) op.getVal(ACTIONS);
- if (actions == null) {
- actions = DEFAULT_ACTIONS;
- opCopy.getDataMap().put(ACTIONS, actions);
- }
-
- // validate that we can load all the actions
- // todo allow creation from blobstore
- for (Map action : actions) {
- if (!action.containsKey(NAME) || !action.containsKey(CLASS)) {
- op.addError("No 'name' or 'class' specified for action: " + action);
- return currentConfig;
- }
- String klass = action.get(CLASS);
- try {
- loader.findClass(klass, TriggerAction.class);
- } catch (Exception e) {
- log.warn("Could not load class : ", e);
- op.addError("Action not found: " + klass + " " + e.getMessage());
- return currentConfig;
- }
- }
- AutoScalingConfig.TriggerConfig trigger = new AutoScalingConfig.TriggerConfig(triggerName, opCopy.getValuesExcluding("name"));
- // validate trigger config
- AutoScaling.Trigger t = null;
- try {
- t = triggerFactory.create(trigger.event, trigger.name, trigger.properties);
- } catch (Exception e) {
- op.addError("Error validating trigger config " + trigger.name + ": " + e.toString());
- return currentConfig;
- } finally {
- if (t != null) {
- IOUtils.closeQuietly(t);
- }
- }
- currentConfig = currentConfig.withTriggerConfig(trigger);
- // check that there's a default SystemLogListener, unless user specified another one
- return withSystemLogListener(currentConfig, triggerName);
- }
-
- private static String fullName = SystemLogListener.class.getName();
- private static String solrName = "solr." + SystemLogListener.class.getSimpleName();
-
- public static AutoScalingConfig withSystemLogListener(AutoScalingConfig autoScalingConfig, String triggerName) {
- Map configs = autoScalingConfig.getTriggerListenerConfigs();
- for (AutoScalingConfig.TriggerListenerConfig cfg : configs.values()) {
- if (triggerName.equals(cfg.trigger)) {
- // already has some listener config
- return autoScalingConfig;
- }
- }
- // need to add
- Map properties = new HashMap<>();
- properties.put(AutoScalingParams.CLASS, SystemLogListener.class.getName());
- properties.put(AutoScalingParams.TRIGGER, triggerName);
- properties.put(AutoScalingParams.STAGE, EnumSet.allOf(TriggerEventProcessorStage.class));
- AutoScalingConfig.TriggerListenerConfig listener =
- new AutoScalingConfig.TriggerListenerConfig(triggerName + CollectionAdminParams.SYSTEM_COLL, properties);
- autoScalingConfig = autoScalingConfig.withTriggerListenerConfig(listener);
- return autoScalingConfig;
- }
-
- private int parseHumanTime(String timeStr) {
- char c = timeStr.charAt(timeStr.length() - 1);
- long timeValue = Long.parseLong(timeStr.substring(0, timeStr.length() - 1));
- int seconds;
- switch (c) {
- case 'h':
- seconds = (int) TimeUnit.HOURS.toSeconds(timeValue);
- break;
- case 'm':
- seconds = (int) TimeUnit.MINUTES.toSeconds(timeValue);
- break;
- case 's':
- seconds = (int) timeValue;
- break;
- default:
- throw new IllegalArgumentException("Invalid time value");
- }
- return seconds;
- }
-
- private AutoScalingConfig handleRemoveTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
- AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
- String triggerName = op.getStr(NAME);
- boolean removeListeners = op.getBoolean(REMOVE_LISTENERS, false);
-
- if (op.hasError()) return currentConfig;
- Map triggerConfigs = currentConfig.getTriggerConfigs();
- if (!triggerConfigs.containsKey(triggerName)) {
- op.addError("No trigger exists with name: " + triggerName);
- return currentConfig;
- }
- triggerConfigs = new HashMap<>(triggerConfigs);
- Set activeListeners = new HashSet<>();
- Map listeners = currentConfig.getTriggerListenerConfigs();
- for (AutoScalingConfig.TriggerListenerConfig listener : listeners.values()) {
- if (triggerName.equals(listener.trigger)) {
- activeListeners.add(listener.name);
- }
- }
- if (!activeListeners.isEmpty()) {
- boolean onlySystemLog = false;
- if (activeListeners.size() == 1) {
- AutoScalingConfig.TriggerListenerConfig cfg = listeners.get(activeListeners.iterator().next());
- if (SystemLogListener.class.getName().equals(cfg.listenerClass) ||
- ("solr." + SystemLogListener.class.getSimpleName()).equals(cfg.listenerClass)) {
- onlySystemLog = true;
- }
- }
- if (removeListeners || onlySystemLog) {
- listeners = new HashMap<>(listeners);
- listeners.keySet().removeAll(activeListeners);
- } else {
- op.addError("Cannot remove trigger: " + triggerName + " because it has active listeners: " + activeListeners);
- return currentConfig;
- }
- }
- triggerConfigs.remove(triggerName);
- currentConfig = currentConfig.withTriggerConfigs(triggerConfigs).withTriggerListenerConfigs(listeners);
- return currentConfig;
- }
-
-
- private boolean setAutoScalingConfig(AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
- verifyAutoScalingConf(currentConfig);
- try {
- cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion());
- } catch (BadVersionException bve) {
- // somebody else has changed the configuration so we must retry
- return false;
- }
- //log.debug("-- saved version " + currentConfig.getZkVersion() + ": " + currentConfig);
- return true;
- }
-
- private void verifyAutoScalingConf(AutoScalingConfig autoScalingConf) throws IOException {
- Policy.Session session = autoScalingConf.getPolicy()
- .createSession(cloudManager);
- log.debug("Verified autoscaling configuration");
- }
-
- @Override
- public String getDescription() {
- return "A handler for autoscaling configuration";
- }
-
- @Override
- public Name getPermissionName(AuthorizationContext request) {
- switch (request.getHttpMethod()) {
- case "GET":
- return Name.AUTOSCALING_READ_PERM;
- case "POST": {
- return StrUtils.splitSmart(request.getResource(), '/', true).size() == 3 ?
- Name.AUTOSCALING_READ_PERM :
- Name.AUTOSCALING_WRITE_PERM;
- }
- default:
- return null;
- }
- }
-
- @Override
- public Collection getApis() {
- return ApiBag.wrapRequestHandlers(this, "autoscaling.Commands");
- }
-
- @Override
- public Boolean registerV2() {
- return Boolean.TRUE;
- }
-
- @Override
- public SolrRequestHandler getSubHandler(String path) {
- if (path.equals("/diagnostics") || path.equals("/suggestions")) return this;
- return null;
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
deleted file mode 100644
index 33bf6b04dab..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.*;
-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.params.AutoScalingParams;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import static org.apache.solr.cloud.autoscaling.TriggerEvent.NODE_NAMES;
-
-/**
- * This class is responsible for using the configured policy and preferences
- * with the hints provided by the trigger event to compute the required cluster operations.
- *
- * The cluster operations computed here are put into the {@link ActionContext}'s properties
- * with the key name "operations". The value is a List of SolrRequest objects.
- */
-public class ComputePlanAction extends TriggerActionBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- // accept all collections by default
- Predicate collectionsPredicate = s -> true;
-
- public ComputePlanAction() {
- super();
- TriggerUtils.validProperties(validProperties, "collections");
- }
-
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
-
- Object value = properties.get("collections");
- if (value instanceof String) {
- String colString = (String) value;
- if (!colString.isEmpty()) {
- List whiteListedCollections = StrUtils.splitSmart(colString, ',');
- collectionsPredicate = whiteListedCollections::contains;
- }
- } else if (value instanceof Map) {
- @SuppressWarnings({"unchecked"})
- Map matchConditions = (Map) value;
- collectionsPredicate = collectionName -> {
- try {
- DocCollection collection = cloudManager.getClusterStateProvider().getCollection(collectionName);
- if (collection == null) {
- log.debug("Collection: {} was not found while evaluating conditions", collectionName);
- return false;
- }
- for (Map.Entry entry : matchConditions.entrySet()) {
- if (!entry.getValue().equals(collection.get(entry.getKey()))) {
- if (log.isDebugEnabled()) {
- log.debug("Collection: {} does not match condition: {}:{}", collectionName, entry.getKey(), entry.getValue());
- }
- return false;
- }
- }
- return true;
- } catch (IOException e) {
- log.error("Exception fetching collection information for: {}", collectionName, e);
- return false;
- }
- };
- }
- }
-
- @Override
- public void process(TriggerEvent event, ActionContext context) throws Exception {
- if (log.isDebugEnabled()) {
- log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
- }
- SolrCloudManager cloudManager = context.getCloudManager();
- try {
- AutoScalingConfig autoScalingConf = cloudManager.getDistribStateManager().getAutoScalingConfig();
- if (autoScalingConf.isEmpty()) {
- throw new Exception("Action: " + getName() + " executed but no policy is configured");
- }
- PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getSession(cloudManager);
- Policy.Session session = sessionWrapper.get();
- ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
- if (log.isTraceEnabled()) {
- log.trace("-- session: {}", session);
- log.trace("-- state: {}", clusterState);
- }
- try {
- Suggester suggester = getSuggester(session, event, context, cloudManager);
- int maxOperations = getMaxNumOps(event, autoScalingConf, clusterState);
- int requestedOperations = getRequestedNumOps(event);
- if (requestedOperations > maxOperations) {
- log.warn("Requested number of operations {} higher than maximum {}, adjusting...",
- requestedOperations, maxOperations);
- }
- int opCount = 0;
- int opLimit = maxOperations;
- if (requestedOperations > 0) {
- opLimit = requestedOperations;
- }
- do {
- // computing changes in large clusters may take a long time
- if (Thread.currentThread().isInterrupted()) {
- throw new InterruptedException("stopping - thread was interrupted");
- }
- @SuppressWarnings({"rawtypes"})
- SolrRequest operation = suggester.getSuggestion();
- opCount++;
- // prepare suggester for the next iteration
- if (suggester.getSession() != null) {
- session = suggester.getSession();
- }
- suggester = getSuggester(session, event, context, cloudManager);
-
- // break on first null op
- // unless a specific number of ops was requested
- // uncomment the following to log too many operations
- /*if (opCount > 10) {
- PolicyHelper.logState(cloudManager, initialSuggester);
- }*/
-
- if (operation == null) {
- if (requestedOperations < 0) {
- //uncomment the following to log zero operations
-// PolicyHelper.logState(cloudManager, initialSuggester);
- break;
- } else {
- log.info("Computed plan empty, remained {} requested ops to try.", opCount - opLimit);
- continue;
- }
- }
- if (log.isDebugEnabled()) {
- log.debug("Computed Plan: {}", operation.getParams());
- }
- Map props = context.getProperties();
- props.compute("operations", (k, v) -> {
- @SuppressWarnings({"unchecked", "rawtypes"})
- List operations = (List) v;
- if (operations == null) operations = new ArrayList<>();
- operations.add(operation);
- return operations;
- });
- } while (opCount < opLimit);
- } finally {
- releasePolicySession(sessionWrapper, session);
- }
- } catch (Exception e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Unexpected exception while processing event: " + event, e);
- }
- }
-
- private void releasePolicySession(PolicyHelper.SessionWrapper sessionWrapper, Policy.Session session) {
- sessionWrapper.returnSession(session);
- sessionWrapper.release();
-
- }
-
- protected int getMaxNumOps(TriggerEvent event, AutoScalingConfig autoScalingConfig, ClusterState clusterState) {
- // estimate a maximum default limit that should be sufficient for most purposes:
- // number of nodes * total number of replicas * 3
- AtomicInteger totalRF = new AtomicInteger();
- clusterState.forEachCollection(coll -> {
- Integer rf = coll.getReplicationFactor();
- if (rf == null) {
- if (coll.getSlices().isEmpty()) {
- rf = 1; // ???
- } else {
- rf = coll.getReplicas().size() / coll.getSlices().size();
- }
- }
- totalRF.addAndGet(rf * coll.getSlices().size());
- });
- int totalMax = clusterState.getLiveNodes().size() * totalRF.get() * 3;
- int maxOp = (Integer) autoScalingConfig.getProperties().getOrDefault(AutoScalingParams.MAX_COMPUTE_OPERATIONS, totalMax);
- Object o = event.getProperty(AutoScalingParams.MAX_COMPUTE_OPERATIONS, maxOp);
- try {
- return Integer.parseInt(String.valueOf(o));
- } catch (Exception e) {
- log.warn("Invalid '{}' event property: {}, using default {}", AutoScalingParams.MAX_COMPUTE_OPERATIONS, o, maxOp);
- return maxOp;
- }
- }
-
- protected int getRequestedNumOps(TriggerEvent event) {
- @SuppressWarnings({"unchecked"})
- Collection ops = (Collection) event.getProperty(TriggerEvent.REQUESTED_OPS, Collections.emptyList());
- if (ops.isEmpty()) {
- return -1;
- } else {
- return ops.size();
- }
- }
-
- private static final String START = "__start__";
-
- protected Suggester getSuggester(Policy.Session session, TriggerEvent event, ActionContext context, SolrCloudManager cloudManager) throws IOException {
- Suggester suggester;
- switch (event.getEventType()) {
- case NODEADDED:
- suggester = getNodeAddedSuggester(cloudManager, session, event);
- break;
- case NODELOST:
- suggester = getNodeLostSuggester(cloudManager, session, event);
- break;
- case SEARCHRATE:
- case METRIC:
- case INDEXSIZE:
- @SuppressWarnings({"unchecked"})
- List ops = (List)event.getProperty(TriggerEvent.REQUESTED_OPS, Collections.emptyList());
- int start = (Integer)event.getProperty(START, 0);
- if (ops.isEmpty() || start >= ops.size()) {
- return NoneSuggester.get(session);
- }
- TriggerEvent.Op op = ops.get(start);
- suggester = session.getSuggester(op.getAction());
- if (suggester instanceof UnsupportedSuggester) {
- @SuppressWarnings({"unchecked"})
- List unsupportedOps = (List)context.getProperties().computeIfAbsent(TriggerEvent.UNSUPPORTED_OPS, k -> new ArrayList());
- unsupportedOps.add(op);
- }
- for (Map.Entry e : op.getHints().entrySet()) {
- suggester = suggester.hint(e.getKey(), e.getValue());
- }
- if (applyCollectionHints(cloudManager, suggester) == 0) return NoneSuggester.get(session);
- suggester = suggester.forceOperation(true);
- event.getProperties().put(START, ++start);
- break;
- case SCHEDULED:
- String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
- CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
- suggester = session.getSuggester(action);
- if (applyCollectionHints(cloudManager, suggester) == 0) return NoneSuggester.get(session);
- break;
- default:
- throw new UnsupportedOperationException("No support for events other than nodeAdded, nodeLost, searchRate, metric, scheduled and indexSize. Received: " + event.getEventType());
- }
- return suggester;
- }
-
- private Suggester getNodeLostSuggester(SolrCloudManager cloudManager, Policy.Session session, TriggerEvent event) throws IOException {
- String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
- CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
- switch (action) {
- case MOVEREPLICA:
- Suggester s = session.getSuggester(action)
- .hint(Suggester.Hint.SRC_NODE, event.getProperty(NODE_NAMES));
- if (applyCollectionHints(cloudManager, s) == 0) return NoneSuggester.get(session);
- return s;
- case DELETENODE:
- int start = (Integer)event.getProperty(START, 0);
- @SuppressWarnings({"unchecked"})
- List srcNodes = (List) event.getProperty(NODE_NAMES);
- if (srcNodes.isEmpty() || start >= srcNodes.size()) {
- return NoneSuggester.get(session);
- }
- String sourceNode = srcNodes.get(start);
- s = session.getSuggester(action)
- .hint(Suggester.Hint.SRC_NODE, event.getProperty(NODE_NAMES));
- if (applyCollectionHints(cloudManager, s) == 0) return NoneSuggester.get(session);
- s.hint(Suggester.Hint.SRC_NODE, Collections.singletonList(sourceNode));
- event.getProperties().put(START, ++start);
- return s;
- case NONE:
- return NoneSuggester.get(session);
- default:
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unsupported preferredOperation: " + action.toLower() + " specified for node lost trigger");
- }
- }
-
- /**
- * Applies collection hints for all collections that match the {@link #collectionsPredicate}
- * and returns the number of collections that matched.
- * @return number of collections that match the {@link #collectionsPredicate}
- * @throws IOException if {@link org.apache.solr.client.solrj.impl.ClusterStateProvider} throws IOException
- */
- private int applyCollectionHints(SolrCloudManager cloudManager, Suggester s) throws IOException {
- ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
- Set set = clusterState.getCollectionStates().keySet().stream()
- .filter(collectionRef -> collectionsPredicate.test(collectionRef))
- .collect(Collectors.toSet());
- if (set.size() < clusterState.getCollectionStates().size()) {
- // apply hints only if a subset of collections are selected
- set.forEach(c -> s.hint(Suggester.Hint.COLL, c));
- }
- return set.size();
- }
-
- private Suggester getNodeAddedSuggester(SolrCloudManager cloudManager, Policy.Session session, TriggerEvent event) throws IOException {
- String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
- Replica.Type replicaType = (Replica.Type) event.getProperty(AutoScalingParams.REPLICA_TYPE, Replica.Type.NRT);
- CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
-
- Suggester suggester = session.getSuggester(action)
- .hint(Suggester.Hint.TARGET_NODE, event.getProperty(NODE_NAMES));
- switch (action) {
- case ADDREPLICA:
- // add all collection/shard pairs and let policy engine figure out which one
- // to place on the target node
- ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
- Set> collShards = new HashSet<>();
- clusterState.getCollectionStates().entrySet().stream()
- .filter(e -> collectionsPredicate.test(e.getKey()))
- .forEach(entry -> {
- DocCollection docCollection = entry.getValue().get();
- if (docCollection != null) {
- docCollection.getActiveSlices().stream()
- .map(slice -> new Pair<>(entry.getKey(), slice.getName()))
- .forEach(collShards::add);
- }
- });
- suggester.hint(Suggester.Hint.COLL_SHARD, collShards);
- suggester.hint(Suggester.Hint.REPLICATYPE, replicaType);
- break;
- case MOVEREPLICA:
- case NONE:
- break;
- default:
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Unsupported preferredOperation=" + preferredOp + " for node added event");
- }
- return suggester;
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
deleted file mode 100644
index 1dfc3b1581c..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
+++ /dev/null
@@ -1,250 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrResponse;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
-import org.apache.solr.client.solrj.cloud.DistribStateManager;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.RequestStatusState;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.TestInjection;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is responsible for executing cluster operations read from the {@link ActionContext}'s properties
- * with the key name "operations".
- */
-public class ExecutePlanAction extends TriggerActionBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- private static final String PREFIX = "op-";
-
- static final int DEFAULT_TASK_TIMEOUT_SECONDS = 120;
- public static final String TASK_TIMEOUT_SECONDS = "taskTimeoutSeconds";
- public static final String TASK_TIMEOUT_FAIL = "taskTimeoutFail";
-
- int taskTimeoutSeconds;
- boolean taskTimeoutFail;
-
- public ExecutePlanAction() {
- TriggerUtils.validProperties(validProperties, TASK_TIMEOUT_SECONDS, TASK_TIMEOUT_FAIL);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- String str = String.valueOf(properties.getOrDefault(TASK_TIMEOUT_SECONDS, DEFAULT_TASK_TIMEOUT_SECONDS));
- taskTimeoutSeconds = Integer.parseInt(str);
- str = String.valueOf(properties.getOrDefault(TASK_TIMEOUT_FAIL, false));
- taskTimeoutFail = Boolean.parseBoolean(str);
- }
-
- @Override
- @SuppressWarnings({"unchecked", "rawtypes"})
- public void process(TriggerEvent event, ActionContext context) throws Exception {
- if (log.isDebugEnabled()) {
- log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
- }
- SolrCloudManager cloudManager = context.getCloudManager();
- List operations = (List) context.getProperty("operations");
- if (operations == null || operations.isEmpty()) {
- log.info("No operations to execute for event: {}", event);
- return;
- }
- try {
- int counter = 0;
- for (SolrRequest operation : operations) {
- if (log.isDebugEnabled()) {
- log.debug("Executing operation: {}", operation.getParams());
- }
- try {
- SolrResponse response = null;
- if (operation instanceof CollectionAdminRequest.AsyncCollectionAdminRequest) {
- CollectionAdminRequest.AsyncCollectionAdminRequest req = (CollectionAdminRequest.AsyncCollectionAdminRequest) operation;
- // waitForFinalState so that the end effects of operations are visible
- req.setWaitForFinalState(true);
- String asyncId = event.getSource() + '/' + event.getId() + '/' + counter;
- String znode = saveAsyncId(cloudManager.getDistribStateManager(), event, asyncId);
- log.trace("Saved requestId: {} in znode: {}", asyncId, znode);
- // TODO: find a better way of using async calls using dataProvider API !!!
- req.setAsyncId(asyncId);
- if (TestInjection.delayInExecutePlanAction != null) {
- cloudManager.getTimeSource().sleep(TestInjection.delayInExecutePlanAction);
- }
- CollectionAdminRequest.RequestStatusResponse statusResponse = null;
- RequestStatusState state = RequestStatusState.FAILED;
- if (!TestInjection.failInExecutePlanAction) {
- SolrResponse asyncResponse = cloudManager.request(req);
- if (asyncResponse.getResponse().get("error") != null) {
- throw new IOException("" + asyncResponse.getResponse().get("error"));
- }
- asyncId = (String)asyncResponse.getResponse().get("requestid");
- statusResponse = waitForTaskToFinish(cloudManager, asyncId,
- taskTimeoutSeconds, TimeUnit.SECONDS);
- }
- if (statusResponse != null) {
- state = statusResponse.getRequestStatus();
- // overwrite to test a long-running task
- if (TestInjection.delayInExecutePlanAction != null &&
- TestInjection.delayInExecutePlanAction > TimeUnit.MILLISECONDS.convert(taskTimeoutSeconds, TimeUnit.SECONDS)) {
- state = RequestStatusState.RUNNING;
- }
- if (TestInjection.failInExecutePlanAction) {
- state = RequestStatusState.FAILED;
- }
- // should we accept partial success here? i.e. some operations won't be completed
- // successfully but the event processing will still be declared a success
- if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED || state == RequestStatusState.NOT_FOUND) {
- // remove pending task marker for this request
- try {
- cloudManager.getDistribStateManager().removeData(znode, -1);
- } catch (Exception e) {
- log.warn("Unexpected exception while trying to delete znode: {}", znode, e);
- }
- }
- response = statusResponse;
- }
- if (state == RequestStatusState.RUNNING || state == RequestStatusState.SUBMITTED) {
- String msg = String.format(Locale.ROOT, "Task %s is still running after " + taskTimeoutSeconds + " seconds. Consider increasing " +
- TASK_TIMEOUT_SECONDS + " action property or `waitFor` of the trigger %s. Operation: %s",
- asyncId, event.source, req);
- if (taskTimeoutFail) {
- throw new IOException(msg);
- } else {
- log.warn(msg);
- }
- } else if (state == RequestStatusState.FAILED) {
- // remove it as a pending task
- try {
- cloudManager.getDistribStateManager().removeData(znode, -1);
- } catch (Exception e) {
- log.warn("Unexpected exception while trying to delete znode: {}", znode, e);
- }
- throw new IOException("Task " + asyncId + " failed: " + (statusResponse != null ? statusResponse : " timed out. Operation: " + req));
- }
- } else {
- // generic response - can't easily determine success or failure
- response = cloudManager.request(operation);
- }
- NamedList result = response.getResponse();
- context.getProperties().compute("responses", (s, o) -> {
- @SuppressWarnings({"unchecked"})
- List> responses = (List>) o;
- if (responses == null) responses = new ArrayList<>(operations.size());
- responses.add(result);
- return responses;
- });
- counter++;
- } catch (IOException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Unexpected exception executing operation: " + operation.getParams(), e);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
- } catch (Exception e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Unexpected exception executing operation: " + operation.getParams(), e);
- }
- }
- } catch (Exception e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Unexpected exception while processing event: " + event, e);
- }
- }
-
-
- static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager cloudManager, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
- long timeoutSeconds = timeUnit.toSeconds(duration);
- RequestStatusState state = RequestStatusState.NOT_FOUND;
- CollectionAdminRequest.RequestStatusResponse statusResponse = null;
- for (int i = 0; i < timeoutSeconds; i++) {
- try {
- statusResponse = (CollectionAdminRequest.RequestStatusResponse)cloudManager.request(CollectionAdminRequest.requestStatus(requestId));
- state = statusResponse.getRequestStatus();
- if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
- log.trace("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
- cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
- return statusResponse;
- } else if (state == RequestStatusState.NOT_FOUND) {
- // the request for this id was never actually submitted! no harm done, just bail out
- log.warn("Task with requestId={} was not found on overseer", requestId);
- cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
- return statusResponse;
- }
- } catch (Exception e) {
- Throwable rootCause = ExceptionUtils.getRootCause(e);
- if (rootCause instanceof IllegalStateException && rootCause.getMessage().contains("Connection pool shut down")) {
- throw e;
- }
- if (rootCause instanceof TimeoutException && rootCause.getMessage().contains("Could not connect to ZooKeeper")) {
- throw e;
- }
- if (rootCause instanceof SolrServerException) {
- throw e;
- }
- log.error("Unexpected Exception while querying status of requestId={}", requestId, e);
- throw e;
- }
- if (i > 0 && i % 5 == 0) {
- log.trace("Task with requestId={} still not complete after {}s. Last state={}", requestId, i * 5, state);
- }
- cloudManager.getTimeSource().sleep(5000);
- }
- log.debug("Task with requestId={} did not complete within {} seconds. Last state={}", timeoutSeconds, requestId, state);
- return statusResponse;
- }
-
- /**
- * Saves the given asyncId in ZK as a persistent sequential node. This allows us to wait for the completion
- * of pending tasks from this event in {@link ScheduledTriggers}
- * before starting the actions of the next event.
- *
- * @return the path of the newly created node in ZooKeeper
- */
- private String saveAsyncId(DistribStateManager stateManager, TriggerEvent event, String asyncId) throws InterruptedException, AlreadyExistsException, IOException, KeeperException {
- String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + event.getSource() + "/" + getName();
- try {
- stateManager.makePath(parentPath);
- } catch (AlreadyExistsException e) {
- // ignore
- }
- return stateManager.createData(parentPath + "/" + PREFIX, Utils.toJSON(Collections.singletonMap("requestid", asyncId)), CreateMode.PERSISTENT_SEQUENTIAL);
- }
-
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
deleted file mode 100644
index 139efe0946f..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-import java.util.StringJoiner;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
-import org.apache.solr.client.solrj.impl.HttpClientUtil;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.PropertiesUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple HTTP callback that POSTs event data to a URL.
- * URL, payload and headers may contain property substitution patterns, with the following properties available:
- *
- * config.* - listener configuration
- * event.* - event properties
- * stage - current stage of event processing
- * actionName - optional current action name
- * context.* - optional {@link ActionContext} properties
- * error - optional error string (from {@link Throwable#toString()})
- * message - optional message
- *
- * The following listener configuration is supported:
- *
- * url - a URL template
- * payload - string, optional payload template. If absent a JSON map of all properties listed above will be used.
- * contentType - string, optional payload content type. If absent then application/json
will be used.
- * header.* - string, optional header template(s). The name of the property without "header." prefix defines the literal header name.
- * timeout - int, optional connection and socket timeout in milliseconds. Default is 60 seconds.
- * followRedirects - boolean, optional setting to follow redirects. Default is false.
- *
- */
-public class HttpTriggerListener extends TriggerListenerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private String urlTemplate;
- private String payloadTemplate;
- private String contentType;
- private Map headerTemplates = new HashMap<>();
- private int timeout = HttpClientUtil.DEFAULT_CONNECT_TIMEOUT;
- private boolean followRedirects;
-
- public HttpTriggerListener() {
- super();
- TriggerUtils.requiredProperties(requiredProperties, validProperties, "url");
- TriggerUtils.validProperties(validProperties, "payload", "contentType", "timeout", "followRedirects");
- validPropertyPrefixes.add("header.");
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
- super.configure(loader, cloudManager, config);
- urlTemplate = (String)config.properties.get("url");
- payloadTemplate = (String)config.properties.get("payload");
- contentType = (String)config.properties.get("contentType");
- config.properties.forEach((k, v) -> {
- if (k.startsWith("header.")) {
- headerTemplates.put(k.substring(7), String.valueOf(v));
- }
- });
- timeout = PropertiesUtil.toInteger(String.valueOf(config.properties.get("timeout")), HttpClientUtil.DEFAULT_CONNECT_TIMEOUT);
- followRedirects = PropertiesUtil.toBoolean(String.valueOf(config.properties.get("followRedirects")));
- }
-
- @Override
- public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context, Throwable error, String message) {
- Properties properties = new Properties();
- properties.setProperty("stage", stage.toString());
- // if configuration used "actionName" but we're in a non-action related stage then PropertiesUtil will
- // throws an exception on missing value - so replace it with an empty string
- if (actionName == null) {
- actionName = "";
- }
- properties.setProperty("actionName", actionName);
- if (context != null) {
- context.getProperties().forEach((k, v) -> {
- properties.setProperty("context." + k, String.valueOf(v));
- });
- }
- if (error != null) {
- properties.setProperty("error", error.toString());
- } else {
- properties.setProperty("error", "");
- }
- if (message != null) {
- properties.setProperty("message", message);
- } else {
- properties.setProperty("message", "");
- }
- // add event properties
- properties.setProperty("event.id", event.getId());
- properties.setProperty("event.source", event.getSource());
- properties.setProperty("event.eventTime", String.valueOf(event.eventTime));
- properties.setProperty("event.eventType", event.getEventType().toString());
- event.getProperties().forEach((k, v) -> {
- properties.setProperty("event.properties." + k, String.valueOf(v));
- });
- // add config properties
- properties.setProperty("config.name", config.name);
- properties.setProperty("config.trigger", config.trigger);
- properties.setProperty("config.listenerClass", config.listenerClass);
- properties.setProperty("config.beforeActions", String.join(",", config.beforeActions));
- properties.setProperty("config.afterActions", String.join(",", config.afterActions));
- StringJoiner joiner = new StringJoiner(",");
- config.stages.forEach(s -> joiner.add(s.toString()));
- properties.setProperty("config.stages", joiner.toString());
- config.properties.forEach((k, v) -> {
- properties.setProperty("config.properties." + k, String.valueOf(v));
- });
- String url = PropertiesUtil.substituteProperty(urlTemplate, properties);
- String payload;
- String type;
- if (payloadTemplate != null) {
- payload = PropertiesUtil.substituteProperty(payloadTemplate, properties);
- if (contentType != null) {
- type = contentType;
- } else {
- type = "application/json";
- }
- } else {
- payload = Utils.toJSONString(properties);
- type = "application/json";
- }
- Map headers = new HashMap<>();
- headerTemplates.forEach((k, v) -> {
- String headerVal = PropertiesUtil.substituteProperty(v, properties);
- if (!headerVal.isEmpty()) {
- headers.put(k, headerVal);
- }
- });
- headers.put("Content-Type", type);
- try {
- cloudManager.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
- } catch (IOException e) {
- log.warn("Exception sending request for event {}", event, e);
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveMarkersPlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveMarkersPlanAction.java
deleted file mode 100644
index 648254203fc..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveMarkersPlanAction.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.client.solrj.cloud.DistribStateManager;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
-import org.apache.solr.client.solrj.cloud.autoscaling.NotEmptyException;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_ACTIVE;
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_STATE;
-
-/**
- * This plan simply removes nodeAdded and nodeLost markers from Zookeeper if their TTL has
- * expired. These markers are used by {@link NodeAddedTrigger} and {@link NodeLostTrigger} to
- * ensure fault tolerance in case of Overseer leader crash.
- */
-public class InactiveMarkersPlanAction extends TriggerActionBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public static final String TTL_PROP = "ttl";
-
- public static final int DEFAULT_TTL_SECONDS = 3600 * 24 * 2;
-
- private int cleanupTTL;
-
- public InactiveMarkersPlanAction() {
- super();
- TriggerUtils.validProperties(validProperties, TTL_PROP);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- String cleanupStr = String.valueOf(properties.getOrDefault(TTL_PROP, String.valueOf(DEFAULT_TTL_SECONDS)));
- try {
- cleanupTTL = Integer.parseInt(cleanupStr);
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "': " + e.toString());
- }
- if (cleanupTTL < 0) {
- throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "', should be > 0. ");
- }
- }
-
- @Override
- public void process(TriggerEvent event, ActionContext context) throws Exception {
- if (log.isTraceEnabled()) {
- log.trace("-- {} cleaning markers", getName());
- }
- // use epoch time to track this across JVMs and nodes
- long currentTimeNs = cloudManager.getTimeSource().getEpochTimeNs();
- Map results = new LinkedHashMap<>();
- Set cleanedUp = new TreeSet<>();
- cleanupMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, currentTimeNs, cleanedUp);
- if (!cleanedUp.isEmpty()) {
- results.put("nodeAdded", cleanedUp);
- cleanedUp = new TreeSet<>();
- }
- cleanupMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, currentTimeNs, cleanedUp);
- if (!cleanedUp.isEmpty()) {
- results.put("nodeLost", cleanedUp);
- }
- if (!results.isEmpty()) {
- context.getProperties().put(getName(), results);
- }
- }
-
- private void cleanupMarkers(String path, long currentTimeNs, Set cleanedUp) throws Exception {
- DistribStateManager stateManager = cloudManager.getDistribStateManager();
- if (!stateManager.hasData(path)) {
- return;
- }
- List markers = stateManager.listData(path);
- markers.forEach(m -> {
- String markerPath = path + "/" + m;
- try {
- Map payload = Utils.getJson(stateManager, markerPath);
- if (payload.isEmpty()) {
- log.trace(" -- ignore {}: either missing or unsupported format", markerPath);
- return;
- }
- boolean activeMarker = payload.getOrDefault(MARKER_STATE, MARKER_ACTIVE)
- .equals(MARKER_ACTIVE);
- long timestamp = ((Number)payload.get("timestamp")).longValue();
- long delta = TimeUnit.NANOSECONDS.toSeconds(currentTimeNs - timestamp);
- if (delta > cleanupTTL || !activeMarker) {
- try {
- stateManager.removeData(markerPath, -1);
- log.trace(" -- remove {}, delta={}, ttl={}, active={}", markerPath, delta, cleanupTTL, activeMarker);
- cleanedUp.add(m);
- } catch (NoSuchElementException nse) {
- // someone already removed it - ignore
- return;
- } catch (BadVersionException be) {
- throw new RuntimeException("should never happen", be);
- } catch (NotEmptyException ne) {
- log.error("Marker znode should be empty but it's not! Ignoring {} ({})", markerPath, ne);
- }
- } else {
- log.trace(" -- keep {}, delta={}, ttl={}, active={}", markerPath, delta, cleanupTTL, activeMarker);
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return;
- } catch (IOException | KeeperException e) {
- log.warn("Could not cleanup marker at {}, skipping... ", markerPath, e);
- }
- });
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java
deleted file mode 100644
index d3de649cf7a..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.cloud.autoscaling;
-
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class checks whether there are shards that have been inactive for a long
- * time (which usually means they are left-overs from shard splitting) and requests their removal
- * after their cleanup TTL period elapsed.
- * Shard delete requests are put into the {@link ActionContext}'s properties
- * with the key name "operations". The value is a List of SolrRequest objects.
- */
-public class InactiveShardPlanAction extends TriggerActionBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public static final String TTL_PROP = "ttl";
-
- public static final int DEFAULT_TTL_SECONDS = 3600 * 24 * 2;
-
- private int cleanupTTL;
-
- public InactiveShardPlanAction() {
- super();
- TriggerUtils.validProperties(validProperties, TTL_PROP);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- String cleanupStr = String.valueOf(properties.getOrDefault(TTL_PROP, String.valueOf(DEFAULT_TTL_SECONDS)));
- try {
- cleanupTTL = Integer.parseInt(cleanupStr);
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "': " + e.toString());
- }
- if (cleanupTTL < 0) {
- throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "', should be > 0. ");
- }
- }
-
- @Override
- public void process(TriggerEvent event, ActionContext context) throws Exception {
- SolrCloudManager cloudManager = context.getCloudManager();
- ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
- Map> cleanup = new LinkedHashMap<>();
- Map> inactive = new LinkedHashMap<>();
- Map> staleLocks = new LinkedHashMap<>();
- state.forEachCollection(coll ->
- coll.getSlices().forEach(s -> {
- if (Slice.State.INACTIVE.equals(s.getState())) {
- inactive.computeIfAbsent(coll.getName(), c -> new ArrayList<>()).add(s.getName());
- String tstampStr = s.getStr(ZkStateReader.STATE_TIMESTAMP_PROP);
- if (tstampStr == null || tstampStr.isEmpty()) {
- return;
- }
- long timestamp = Long.parseLong(tstampStr);
- // this timestamp uses epoch time
- long currentTime = cloudManager.getTimeSource().getEpochTimeNs();
- long delta = TimeUnit.NANOSECONDS.toSeconds(currentTime - timestamp);
- if (log.isDebugEnabled()) {
- log.debug("{}/{}: tstamp={}, time={}, delta={}", coll.getName(), s.getName(), timestamp, currentTime, delta);
- }
- if (delta > cleanupTTL) {
- if (log.isDebugEnabled()) {
- log.debug("-- delete inactive {} / {}", coll.getName(), s.getName());
- }
- @SuppressWarnings({"unchecked", "rawtypes"})
- List operations = (List)context.getProperties().computeIfAbsent("operations", k -> new ArrayList<>());
- operations.add(CollectionAdminRequest.deleteShard(coll.getName(), s.getName()));
- cleanup.computeIfAbsent(coll.getName(), c -> new ArrayList<>()).add(s.getName());
- }
- }
- // check for stale shard split locks
- String parentPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll.getName();
- List locks;
- try {
- locks = cloudManager.getDistribStateManager().listData(parentPath).stream()
- .filter(name -> name.endsWith("-splitting"))
- .collect(Collectors.toList());
- for (String lock : locks) {
- try {
- String lockPath = parentPath + "/" + lock;
- Map lockData = Utils.getJson(cloudManager.getDistribStateManager(), lockPath);
- String tstampStr = (String)lockData.get(ZkStateReader.STATE_TIMESTAMP_PROP);
- if (tstampStr == null || tstampStr.isEmpty()) {
- return;
- }
- long timestamp = Long.parseLong(tstampStr);
- // this timestamp uses epoch time
- long currentTime = cloudManager.getTimeSource().getEpochTimeNs();
- long delta = TimeUnit.NANOSECONDS.toSeconds(currentTime - timestamp);
- if (log.isDebugEnabled()) {
- log.debug("{}/{}: locktstamp={}, time={}, delta={}", coll.getName(), lock, timestamp, currentTime, delta);
- }
- if (delta > cleanupTTL) {
- if (log.isDebugEnabled()) {
- log.debug("-- delete inactive split lock for {}/{}, delta={}", coll.getName(), lock, delta);
- }
- cloudManager.getDistribStateManager().removeData(lockPath, -1);
- lockData.put("currentTimeNs", currentTime);
- lockData.put("deltaSec", delta);
- lockData.put("ttlSec", cleanupTTL);
- staleLocks.put(coll.getName() + "/" + lock, lockData);
- } else {
- if (log.isDebugEnabled()) {
- log.debug("-- lock {}/{} still active (delta={})", coll.getName(), lock, delta);
- }
- }
- } catch (NoSuchElementException nse) {
- // already removed by someone else - ignore
- }
- }
- } catch (Exception e) {
- log.warn("Exception checking for inactive shard split locks in {}", parentPath, e);
- }
- })
- );
- Map results = new LinkedHashMap<>();
- if (!cleanup.isEmpty()) {
- results.put("inactive", inactive);
- results.put("cleanup", cleanup);
- }
- if (!staleLocks.isEmpty()) {
- results.put("staleLocks", staleLocks);
- }
- if (!results.isEmpty()) {
- context.getProperties().put(getName(), results);
- }
- }
-}
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
deleted file mode 100644
index e9878281dec..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
+++ /dev/null
@@ -1,562 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.Locale;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-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.params.CollectionParams;
-import org.apache.solr.common.params.CommonAdminParams;
-import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.metrics.SolrCoreMetricManager;
-import org.apache.solr.update.SolrIndexSplitter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.CORE_IDX;
-
-/**
- *
- */
-public class IndexSizeTrigger extends TriggerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- // configuration properties
- public static final String ABOVE_BYTES_PROP = "aboveBytes";
- public static final String ABOVE_DOCS_PROP = "aboveDocs";
- public static final String ABOVE_OP_PROP = "aboveOp";
- public static final String BELOW_BYTES_PROP = "belowBytes";
- public static final String BELOW_DOCS_PROP = "belowDocs";
- public static final String BELOW_OP_PROP = "belowOp";
- public static final String COLLECTIONS_PROP = "collections";
- public static final String MAX_OPS_PROP = "maxOps";
- public static final String SPLIT_FUZZ_PROP = CommonAdminParams.SPLIT_FUZZ;
- public static final String SPLIT_METHOD_PROP = CommonAdminParams.SPLIT_METHOD;
- public static final String SPLIT_BY_PREFIX = CommonAdminParams.SPLIT_BY_PREFIX;
-
- // event properties
- public static final String BYTES_SIZE_KEY = "__bytes__";
- public static final String TOTAL_BYTES_SIZE_KEY = "__total_bytes__";
- public static final String DOCS_SIZE_KEY = "__docs__";
- public static final String MAX_DOC_KEY = "__maxDoc__";
- public static final String COMMIT_SIZE_KEY = "__commitBytes__";
- public static final String ABOVE_SIZE_KEY = "aboveSize";
- public static final String BELOW_SIZE_KEY = "belowSize";
- public static final String VIOLATION_KEY = "violationType";
-
- public static final int DEFAULT_MAX_OPS = 10;
-
- public enum Unit { bytes, docs }
-
- private long aboveBytes, aboveDocs, belowBytes, belowDocs;
- private int maxOps;
- private SolrIndexSplitter.SplitMethod splitMethod;
- private boolean splitByPrefix;
- private float splitFuzz;
- private CollectionParams.CollectionAction aboveOp, belowOp;
- private final Set collections = new HashSet<>();
- private final Map lastAboveEventMap = new ConcurrentHashMap<>();
- private final Map lastBelowEventMap = new ConcurrentHashMap<>();
-
- public IndexSizeTrigger(String name) {
- super(TriggerEventType.INDEXSIZE, name);
- TriggerUtils.validProperties(validProperties,
- ABOVE_BYTES_PROP, ABOVE_DOCS_PROP, ABOVE_OP_PROP,
- BELOW_BYTES_PROP, BELOW_DOCS_PROP, BELOW_OP_PROP,
- COLLECTIONS_PROP, MAX_OPS_PROP,
- SPLIT_METHOD_PROP, SPLIT_FUZZ_PROP, SPLIT_BY_PREFIX);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- String aboveStr = String.valueOf(properties.getOrDefault(ABOVE_BYTES_PROP, Long.MAX_VALUE));
- String belowStr = String.valueOf(properties.getOrDefault(BELOW_BYTES_PROP, -1));
- try {
- aboveBytes = Long.parseLong(aboveStr);
- if (aboveBytes <= 0) {
- throw new Exception("value must be > 0");
- }
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), ABOVE_BYTES_PROP, "invalid value '" + aboveStr + "': " + e.toString());
- }
- try {
- belowBytes = Long.parseLong(belowStr);
- if (belowBytes < 0) {
- belowBytes = -1;
- }
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), BELOW_BYTES_PROP, "invalid value '" + belowStr + "': " + e.toString());
- }
- // below must be at least 2x smaller than above, otherwise splitting a shard
- // would immediately put the shard below the threshold and cause the mergeshards action
- if (belowBytes > 0 && (belowBytes * 2 > aboveBytes)) {
- throw new TriggerValidationException(getName(), BELOW_BYTES_PROP,
- "invalid value " + belowBytes + ", should be less than half of '" + ABOVE_BYTES_PROP + "' value, which is " + aboveBytes);
- }
- // do the same for docs bounds
- aboveStr = String.valueOf(properties.getOrDefault(ABOVE_DOCS_PROP, Long.MAX_VALUE));
- belowStr = String.valueOf(properties.getOrDefault(BELOW_DOCS_PROP, -1));
- try {
- aboveDocs = Long.parseLong(aboveStr);
- if (aboveDocs <= 0) {
- throw new Exception("value must be > 0");
- }
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), ABOVE_DOCS_PROP, "invalid value '" + aboveStr + "': " + e.toString());
- }
- try {
- belowDocs = Long.parseLong(belowStr);
- if (belowDocs < 0) {
- belowDocs = -1;
- }
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), BELOW_DOCS_PROP, "invalid value '" + belowStr + "': " + e.toString());
- }
- // below must be at least 2x smaller than above, otherwise splitting a shard
- // would immediately put the shard below the threshold and cause the mergeshards action
- if (belowDocs > 0 && (belowDocs * 2 > aboveDocs)) {
- throw new TriggerValidationException(getName(), BELOW_DOCS_PROP,
- "invalid value " + belowDocs + ", should be less than half of '" + ABOVE_DOCS_PROP + "' value, which is " + aboveDocs);
- }
-
- String collectionsString = (String) properties.get(COLLECTIONS_PROP);
- if (collectionsString != null && !collectionsString.isEmpty()) {
- collections.addAll(StrUtils.splitSmart(collectionsString, ','));
- }
- String aboveOpStr = String.valueOf(properties.getOrDefault(ABOVE_OP_PROP, CollectionParams.CollectionAction.SPLITSHARD.toLower()));
- // TODO: this is a placeholder until SOLR-9407 is implemented
- String belowOpStr = String.valueOf(properties.getOrDefault(BELOW_OP_PROP, CollectionParams.CollectionAction.MERGESHARDS.toLower()));
- aboveOp = CollectionParams.CollectionAction.get(aboveOpStr);
- if (aboveOp == null) {
- throw new TriggerValidationException(getName(), ABOVE_OP_PROP, "unrecognized value of: '" + aboveOpStr + "'");
- }
- belowOp = CollectionParams.CollectionAction.get(belowOpStr);
- if (belowOp == null) {
- throw new TriggerValidationException(getName(), BELOW_OP_PROP, "unrecognized value of: '" + belowOpStr + "'");
- }
- String maxOpsStr = String.valueOf(properties.getOrDefault(MAX_OPS_PROP, DEFAULT_MAX_OPS));
- try {
- maxOps = Integer.parseInt(maxOpsStr);
- if (maxOps < 1) {
- throw new Exception("must be > 1");
- }
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), MAX_OPS_PROP, "invalid value: '" + maxOpsStr + "': " + e.getMessage());
- }
- String methodStr = (String)properties.getOrDefault(SPLIT_METHOD_PROP, SolrIndexSplitter.SplitMethod.LINK.toLower());
- splitMethod = SolrIndexSplitter.SplitMethod.get(methodStr);
- if (splitMethod == null) {
- throw new TriggerValidationException(getName(), SPLIT_METHOD_PROP, "unrecognized value of: '" + methodStr + "'");
- }
- String fuzzStr = String.valueOf(properties.getOrDefault(SPLIT_FUZZ_PROP, 0.0f));
- try {
- splitFuzz = Float.parseFloat(fuzzStr);
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), SPLIT_FUZZ_PROP, "invalid value: '" + fuzzStr + "': " + e.getMessage());
- }
- String splitByPrefixStr = String.valueOf(properties.getOrDefault(SPLIT_BY_PREFIX, false));
- try {
- splitByPrefix = getValidBool(splitByPrefixStr);
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), SPLIT_BY_PREFIX, "invalid value: '" + splitByPrefixStr + "': " + e.getMessage());
- }
- }
-
- private boolean getValidBool(String str) throws Exception {
- if (str != null && (str.toLowerCase(Locale.ROOT).equals("true") || str.toLowerCase(Locale.ROOT).equals("false"))) {
- return Boolean.parseBoolean(str);
- }
- throw new IllegalArgumentException("Expected a valid boolean value but got " + str);
- }
-
- @Override
- protected Map getState() {
- Map state = new HashMap<>();
- state.put("lastAboveEventMap", lastAboveEventMap);
- state.put("lastBelowEventMap", lastBelowEventMap);
- return state;
- }
-
- @Override
- @SuppressWarnings({"unchecked"})
- protected void setState(Map state) {
- this.lastAboveEventMap.clear();
- this.lastBelowEventMap.clear();
- Map replicaVsTime = (Map)state.get("lastAboveEventMap");
- if (replicaVsTime != null) {
- this.lastAboveEventMap.putAll(replicaVsTime);
- }
- replicaVsTime = (Map)state.get("lastBelowEventMap");
- if (replicaVsTime != null) {
- this.lastBelowEventMap.putAll(replicaVsTime);
- }
- }
-
- @Override
- public void restoreState(AutoScaling.Trigger old) {
- assert old.isClosed();
- if (old instanceof IndexSizeTrigger) {
- IndexSizeTrigger that = (IndexSizeTrigger)old;
- assert this.name.equals(that.name);
- this.lastAboveEventMap.clear();
- this.lastBelowEventMap.clear();
- this.lastAboveEventMap.putAll(that.lastAboveEventMap);
- this.lastBelowEventMap.putAll(that.lastBelowEventMap);
- } else {
- throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
- "Unable to restore state from an unknown type of trigger");
- }
- }
-
- @Override
- @SuppressWarnings({"unchecked", "rawtypes"})
- public void run() {
- synchronized(this) {
- if (isClosed) {
- log.warn("{} ran but was already closed", getName());
- return;
- }
- }
- AutoScaling.TriggerEventProcessor processor = processorRef.get();
- if (processor == null) {
- return;
- }
-
- // replica name / info + size, retrieved from leaders only
- Map currentSizes = new HashMap<>();
-
- try {
- ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
- for (String node : clusterState.getLiveNodes()) {
- Map metricTags = new HashMap<>();
- // coll, shard, replica
- Map>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
- infos.forEach((coll, shards) -> {
- if (!collections.isEmpty() && !collections.contains(coll)) {
- return;
- }
- DocCollection docCollection = clusterState.getCollection(coll);
-
- shards.forEach((sh, replicas) -> {
- // check only the leader replica in an active shard
- Slice s = docCollection.getSlice(sh);
- if (s.getState() != Slice.State.ACTIVE) {
- return;
- }
- Replica r = s.getLeader();
- // no leader - don't do anything
- if (r == null) {
- return;
- }
- // not on this node
- if (!r.getNodeName().equals(node)) {
- return;
- }
- // find ReplicaInfo
- Replica info = null;
- for (Replica ri : replicas) {
- if (r.getCoreName().equals(ri.getCoreName())) {
- info = ri;
- break;
- }
- }
- if (info == null) {
- // probably replica is not on this node?
- return;
- }
- // we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
- String replicaName = Utils.parseMetricsReplicaName(coll, info.getCoreName());
- if (replicaName == null) { // should never happen???
- replicaName = info.getName(); // which is actually coreNode name...
- }
- String registry = SolrCoreMetricManager.createRegistryName(true, coll, sh, replicaName, null);
- String tag = "metrics:" + registry + ":" + CORE_IDX.metricsAttribute;
- metricTags.put(tag, info);
- tag = "metrics:" + registry + ":SEARCHER.searcher.numDocs";
- metricTags.put(tag, info);
- tag = "metrics:" + registry + ":SEARCHER.searcher.maxDoc";
- metricTags.put(tag, info);
- tag = "metrics:" + registry + ":SEARCHER.searcher.indexCommitSize";
- metricTags.put(tag, info);
- });
- });
- if (metricTags.isEmpty()) {
- continue;
- }
- Map sizes = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
- sizes.forEach((tag, size) -> {
- final Replica info = metricTags.get(tag);
- if (info == null) {
- log.warn("Missing replica info for response tag {}", tag);
- } else {
- // verify that it's a Number
- if (!(size instanceof Number)) {
- log.warn("invalid size value for tag {} - not a number: '{}' is {}", tag, size, size.getClass().getName());
- return;
- }
-
- Replica currentInfo = currentSizes.computeIfAbsent(info.getCoreName(), k -> (Replica) info.clone());
- if (tag.contains("INDEX")) {
- currentInfo.getProperties().put(TOTAL_BYTES_SIZE_KEY, ((Number) size).longValue());
- } else if (tag.endsWith("SEARCHER.searcher.numDocs")) {
- currentInfo.getProperties().put(DOCS_SIZE_KEY, ((Number) size).longValue());
- } else if (tag.endsWith("SEARCHER.searcher.maxDoc")) {
- currentInfo.getProperties().put(MAX_DOC_KEY, ((Number) size).longValue());
- } else if (tag.endsWith("SEARCHER.searcher.indexCommitSize")) {
- currentInfo.getProperties().put(COMMIT_SIZE_KEY, ((Number) size).longValue());
- }
- }
- });
- }
- } catch (IOException e) {
- log.warn("Error running trigger {}", getName(), e);
- return;
- }
-
- long now = cloudManager.getTimeSource().getTimeNs();
-
- // now check thresholds
-
- // collection / list(info)
- Map> aboveSize = new HashMap<>();
-
- Set splittable = new HashSet<>();
-
- currentSizes.forEach((coreName, info) -> {
- // calculate estimated bytes
- long maxDoc = (Long)info.get(MAX_DOC_KEY);
- long numDocs = (Long)info.get(DOCS_SIZE_KEY);
- long commitSize = (Long)info.get(COMMIT_SIZE_KEY, 0L);
- if (commitSize <= 0) {
- commitSize = (Long)info.get(TOTAL_BYTES_SIZE_KEY);
- }
- // calculate estimated size as a side-effect
- commitSize = estimatedSize(maxDoc, numDocs, commitSize);
- info.getProperties().put(BYTES_SIZE_KEY, commitSize);
-
- if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes ||
- (Long)info.get(DOCS_SIZE_KEY) > aboveDocs) {
- if (waitForElapsed(coreName, now, lastAboveEventMap)) {
- List infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
- if (!infos.contains(info)) {
- if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes) {
- info.getProperties().put(VIOLATION_KEY, ABOVE_BYTES_PROP);
- } else {
- info.getProperties().put(VIOLATION_KEY, ABOVE_DOCS_PROP);
- }
- infos.add(info);
- splittable.add(info.getName());
- }
- }
- } else {
- // no violation - clear waitForElapsed
- lastAboveEventMap.remove(coreName);
- }
- });
-
- // collection / list(info)
- Map> belowSize = new HashMap<>();
-
- currentSizes.forEach((coreName, info) -> {
- if (((Long)info.get(BYTES_SIZE_KEY) < belowBytes ||
- (Long)info.get(DOCS_SIZE_KEY) < belowDocs) &&
- // make sure we don't produce conflicting ops
- !splittable.contains(info.getName())) {
- if (waitForElapsed(coreName, now, lastBelowEventMap)) {
- List infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
- if (!infos.contains(info)) {
- if ((Long)info.get(BYTES_SIZE_KEY) < belowBytes) {
- info.getProperties().put(VIOLATION_KEY, BELOW_BYTES_PROP);
- } else {
- info.getProperties().put(VIOLATION_KEY, BELOW_DOCS_PROP);
- }
- infos.add(info);
- }
- }
- } else {
- // no violation - clear waitForElapsed
- lastBelowEventMap.remove(coreName);
- }
- });
-
- if (aboveSize.isEmpty() && belowSize.isEmpty()) {
- log.trace("NO VIOLATIONS: Now={}", now);
- log.trace("lastAbove={}", lastAboveEventMap);
- log.trace("lastBelow={}", lastBelowEventMap);
- return;
- }
-
- // find the earliest time when a condition was exceeded
- final AtomicLong eventTime = new AtomicLong(now);
-
- // calculate ops
- final List ops = new ArrayList<>();
- aboveSize.forEach((coll, replicas) -> {
- // sort by decreasing size to first split the largest ones
- // XXX see the comment below about using DOCS_SIZE_PROP in lieu of BYTES_SIZE_PROP
- replicas.sort((r1, r2) -> {
- long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
- if (delta > 0) {
- return -1;
- } else if (delta < 0) {
- return 1;
- } else {
- return 0;
- }
- });
- replicas.forEach(r -> {
- if (ops.size() >= maxOps) {
- return;
- }
- TriggerEvent.Op op = new TriggerEvent.Op(aboveOp);
- op.addHint(Suggester.Hint.COLL_SHARD, new Pair<>(coll, r.getShard()));
- Map params = new HashMap<>();
- params.put(SPLIT_METHOD_PROP, splitMethod.toLower());
- if (splitFuzz > 0) {
- params.put(SPLIT_FUZZ_PROP, splitFuzz);
- }
- params.put(SPLIT_BY_PREFIX, splitByPrefix);
- op.addHint(Suggester.Hint.PARAMS, params);
- ops.add(op);
- Long time = lastAboveEventMap.get(r.getCoreName());
- if (time != null && eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- });
- belowSize.forEach((coll, replicas) -> {
- if (replicas.size() < 2) {
- return;
- }
- if (ops.size() >= maxOps) {
- return;
- }
- // sort by increasing size
- replicas.sort((r1, r2) -> {
- // XXX this is not quite correct - if BYTES_SIZE_PROP decided that replica got here
- // then we should be sorting by BYTES_SIZE_PROP. However, since DOCS and BYTES are
- // loosely correlated it's simpler to sort just by docs (which better reflects the "too small"
- // condition than index size, due to possibly existing deleted docs that still occupy space)
- long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
- if (delta > 0) {
- return 1;
- } else if (delta < 0) {
- return -1;
- } else {
- return 0;
- }
- });
-
- // TODO: MERGESHARDS is not implemented yet. For now take the top two smallest shards
- // TODO: but in the future we probably need to get ones with adjacent ranges.
-
- // TODO: generate as many MERGESHARDS as needed to consume all belowSize shards
- TriggerEvent.Op op = new TriggerEvent.Op(belowOp);
- op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(0).getShard()));
- op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(1).getShard()));
- ops.add(op);
- Long time = lastBelowEventMap.get(replicas.get(0).getCoreName());
- if (time != null && eventTime.get() > time) {
- eventTime.set(time);
- }
- time = lastBelowEventMap.get(replicas.get(1).getCoreName());
- if (time != null && eventTime.get() > time) {
- eventTime.set(time);
- }
- });
-
- if (ops.isEmpty()) {
- return;
- }
- if (processor.process(new IndexSizeEvent(getName(), eventTime.get(), ops, aboveSize, belowSize))) {
- // update last event times
- aboveSize.forEach((coll, replicas) -> {
- replicas.forEach(r -> lastAboveEventMap.put(r.getCoreName(), now));
- });
- belowSize.forEach((coll, replicas) -> {
- if (replicas.size() < 2) {
- return;
- }
- lastBelowEventMap.put(replicas.get(0).getCoreName(), now);
- lastBelowEventMap.put(replicas.get(1).getCoreName(), now);
- });
- }
- }
-
- public static long estimatedSize(long maxDoc, long numDocs, long commitSize) {
- if (maxDoc == 0) {
- return 0;
- }
- if (maxDoc == numDocs) {
- return commitSize;
- }
- return commitSize * numDocs / maxDoc;
- }
-
- private boolean waitForElapsed(String name, long now, Map lastEventMap) {
- Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
- long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
- log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
- if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
- return false;
- }
- return true;
- }
-
- public static class IndexSizeEvent extends TriggerEvent {
- public IndexSizeEvent(String source, long eventTime, List ops, Map> aboveSize,
- Map> belowSize) {
- super(TriggerEventType.INDEXSIZE, source, eventTime, null);
- properties.put(TriggerEvent.REQUESTED_OPS, ops);
- // avoid passing very large amounts of data here - just use replica names
- TreeMap above = new TreeMap<>();
- aboveSize.forEach((coll, replicas) ->
- replicas.forEach(r -> above.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
- properties.put(ABOVE_SIZE_KEY, above);
- TreeMap below = new TreeMap<>();
- belowSize.forEach((coll, replicas) ->
- replicas.forEach(r -> below.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
- properties.put(BELOW_SIZE_KEY, below);
- }
- }
-
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/LoggingListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/LoggingListener.java
deleted file mode 100644
index a7dcf63c81f..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/LoggingListener.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.lang.invoke.MethodHandles;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implementation of {@link TriggerListener} that reports
- * events to a log.
- */
-public class LoggingListener extends TriggerListenerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- @Override
- public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
- Throwable error, String message) {
- log.info("{}: stage={}, actionName={}, event={}, error={}, messsage={}", config.name, stage, actionName, event, error, message);
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
deleted file mode 100644
index 573ac77bd6b..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-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.Slice;
-import org.apache.solr.common.params.AutoScalingParams;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.Pair;
-import org.apache.solr.core.SolrResourceLoader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.params.AutoScalingParams.ABOVE;
-import static org.apache.solr.common.params.AutoScalingParams.BELOW;
-import static org.apache.solr.common.params.AutoScalingParams.METRIC;
-import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
-
-public class MetricTrigger extends TriggerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private String metric;
- private Number above, below;
- private String collection, shard, node, preferredOp;
-
- private final Map lastNodeEvent = new ConcurrentHashMap<>();
-
- public MetricTrigger(String name) {
- super(TriggerEventType.METRIC, name);
- TriggerUtils.requiredProperties(requiredProperties, validProperties, METRIC);
- TriggerUtils.validProperties(validProperties, ABOVE, BELOW, PREFERRED_OP,
- AutoScalingParams.COLLECTION,
- AutoScalingParams.SHARD,
- AutoScalingParams.NODE);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- this.metric = (String) properties.get(METRIC);
- this.above = (Number) properties.get(ABOVE);
- this.below = (Number) properties.get(BELOW);
- this.collection = (String) properties.getOrDefault(AutoScalingParams.COLLECTION, Policy.ANY);
- shard = (String) properties.getOrDefault(AutoScalingParams.SHARD, Policy.ANY);
- if (collection.equals(Policy.ANY) && !shard.equals(Policy.ANY)) {
- throw new TriggerValidationException("shard", "When 'shard' is other than #ANY then collection name must be also other than #ANY");
- }
- node = (String) properties.getOrDefault(AutoScalingParams.NODE, Policy.ANY);
- preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
- }
-
- @Override
- protected Map getState() {
- return null;
- }
-
- @Override
- protected void setState(Map state) {
- lastNodeEvent.clear();
- @SuppressWarnings({"unchecked"})
- Map nodeTimes = (Map) state.get("lastNodeEvent");
- if (nodeTimes != null) {
- lastNodeEvent.putAll(nodeTimes);
- }
- }
-
- @Override
- public void restoreState(AutoScaling.Trigger old) {
- assert old.isClosed();
- if (old instanceof MetricTrigger) {
- MetricTrigger that = (MetricTrigger) old;
- assert this.name.equals(that.name);
- this.lastNodeEvent.clear();
- this.lastNodeEvent.putAll(that.lastNodeEvent);
- } else {
- throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
- "Unable to restore state from an unknown type of trigger");
- }
- }
-
- @Override
- public void run() {
- AutoScaling.TriggerEventProcessor processor = processorRef.get();
- if (processor == null) {
- return;
- }
-
- Set liveNodes = null;
- if (node.equals(Policy.ANY)) {
- if (collection.equals(Policy.ANY)) {
- liveNodes = cloudManager.getClusterStateProvider().getLiveNodes();
- } else {
- final Set nodes = new HashSet<>();
- ClusterState.CollectionRef ref = cloudManager.getClusterStateProvider().getState(collection);
- DocCollection docCollection;
- if (ref == null || (docCollection = ref.get()) == null) {
- log.warn("MetricTrigger could not find collection: {}", collection);
- return;
- }
- if (shard.equals(Policy.ANY)) {
- docCollection.getReplicas().forEach(replica -> {
- nodes.add(replica.getNodeName());
- });
- } else {
- Slice slice = docCollection.getSlice(shard);
- if (slice == null) {
- log.warn("MetricTrigger could not find collection: {} shard: {}", collection, shard);
- return;
- }
- slice.getReplicas().forEach(replica -> nodes.add(replica.getNodeName()));
- }
- liveNodes = nodes;
- }
- } else {
- liveNodes = Collections.singleton(node);
- }
-
- Map rates = new HashMap<>(liveNodes.size());
- for (String node : liveNodes) {
- Map values = cloudManager.getNodeStateProvider().getNodeValues(node, Collections.singletonList(metric));
- values.forEach((tag, rate) -> rates.computeIfAbsent(node, s -> (Number) rate));
- }
-
- long now = cloudManager.getTimeSource().getTimeNs();
- // check for exceeded rates and filter out those with less than waitFor from previous events
- Map hotNodes = rates.entrySet().stream()
- .filter(entry -> waitForElapsed(entry.getKey(), now, lastNodeEvent))
- .filter(entry -> (below != null && Double.compare(entry.getValue().doubleValue(), below.doubleValue()) < 0) || (above != null && Double.compare(entry.getValue().doubleValue(), above.doubleValue()) > 0))
- .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
-
- if (hotNodes.isEmpty()) return;
-
- final AtomicLong eventTime = new AtomicLong(now);
- hotNodes.forEach((n, r) -> {
- long time = lastNodeEvent.get(n);
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
-
- if (processor.process(new MetricBreachedEvent(getName(), collection, shard, preferredOp, eventTime.get(), metric, hotNodes))) {
- hotNodes.keySet().forEach(node -> lastNodeEvent.put(node, now));
- }
- }
-
- private boolean waitForElapsed(String name, long now, Map lastEventMap) {
- Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
- long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
- log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
- if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
- return false;
- }
- return true;
- }
-
- public static class MetricBreachedEvent extends TriggerEvent {
- public MetricBreachedEvent(String source, String collection, String shard, String preferredOp, long eventTime, String metric, Map hotNodes) {
- super(TriggerEventType.METRIC, source, eventTime, null);
- properties.put(METRIC, metric);
- properties.put(AutoScalingParams.NODE, hotNodes);
- if (!collection.equals(Policy.ANY)) {
- properties.put(AutoScalingParams.COLLECTION, collection);
- }
- if (!shard.equals(Policy.ANY)) {
- properties.put(AutoScalingParams.SHARD, shard);
- }
- properties.put(PREFERRED_OP, preferredOp);
-
- // specify requested ops
- List ops = new ArrayList<>(hotNodes.size());
- for (String n : hotNodes.keySet()) {
- Op op = new Op(CollectionParams.CollectionAction.get(preferredOp));
- op.addHint(Suggester.Hint.SRC_NODE, n);
- if (!collection.equals(Policy.ANY)) {
- if (!shard.equals(Policy.ANY)) {
- op.addHint(Suggester.Hint.COLL_SHARD, new Pair<>(collection, shard));
- } else {
- op.addHint(Suggester.Hint.COLL, collection);
- }
- }
- ops.add(op);
- }
- properties.put(TriggerEvent.REQUESTED_OPS, ops);
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
deleted file mode 100644
index 42188e4587a..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
+++ /dev/null
@@ -1,250 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_ACTIVE;
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_INACTIVE;
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_STATE;
-import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
-import static org.apache.solr.common.params.AutoScalingParams.REPLICA_TYPE;
-
-/**
- * Trigger for the {@link TriggerEventType#NODEADDED} event
- */
-public class NodeAddedTrigger extends TriggerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private Set lastLiveNodes = new HashSet<>();
-
- private Map nodeNameVsTimeAdded = new HashMap<>();
-
- private String preferredOp;
- private Replica.Type replicaType = Replica.Type.NRT;
-
- public NodeAddedTrigger(String name) {
- super(TriggerEventType.NODEADDED, name);
- TriggerUtils.validProperties(validProperties, PREFERRED_OP, REPLICA_TYPE);
- }
-
- @Override
- public void init() throws Exception {
- super.init();
- lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
- log.debug("NodeAddedTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
- log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
- // pick up added nodes for which marker paths were created
- try {
- List added = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
- added.forEach(n -> {
- String markerPath = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + n;
- try {
- Map markerData = Utils.getJson(stateManager, markerPath);
- // skip inactive markers
- if (markerData.getOrDefault(MARKER_STATE, MARKER_ACTIVE).equals(MARKER_INACTIVE)) {
- return;
- }
- } catch (InterruptedException | IOException | KeeperException e) {
- log.debug("-- ignoring marker {} state due to error{}", markerPath, e);
- }
- // don't add nodes that have since gone away
- if (lastLiveNodes.contains(n) && !nodeNameVsTimeAdded.containsKey(n)) {
- // since {@code #restoreState(AutoScaling.Trigger)} is called first, the timeAdded for a node may also be restored
- log.debug("Adding node from marker path: {}", n);
- nodeNameVsTimeAdded.put(n, cloudManager.getTimeSource().getTimeNs());
- }
- });
- } catch (NoSuchElementException e) {
- // ignore
- } catch (Exception e) {
- log.warn("Exception retrieving nodeLost markers", e);
- }
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
- preferredOp = preferredOp.toLowerCase(Locale.ROOT);
- String replicaTypeStr = (String) properties.getOrDefault(REPLICA_TYPE, Replica.Type.NRT.name());
- // verify
- try {
- replicaType = Replica.Type.valueOf(replicaTypeStr);
- } catch (IllegalArgumentException | NullPointerException e) {
- throw new TriggerValidationException("Unsupported replicaType=" + replicaTypeStr + " specified for node added trigger");
- }
-
- CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
- switch (action) {
- case ADDREPLICA:
- case MOVEREPLICA:
- case NONE:
- break;
- default:
- throw new TriggerValidationException("Unsupported preferredOperation=" + preferredOp + " specified for node added trigger");
- }
- }
-
- @Override
- public void restoreState(AutoScaling.Trigger old) {
- assert old.isClosed();
- if (old instanceof NodeAddedTrigger) {
- NodeAddedTrigger that = (NodeAddedTrigger) old;
- assert this.name.equals(that.name);
- this.lastLiveNodes.clear();
- this.lastLiveNodes.addAll(that.lastLiveNodes);
- this.nodeNameVsTimeAdded.clear();
- this.nodeNameVsTimeAdded.putAll(that.nodeNameVsTimeAdded);
- } else {
- throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
- "Unable to restore state from an unknown type of trigger");
- }
- }
-
- @Override
- protected Map getState() {
- Map state = new HashMap<>();
- state.put("lastLiveNodes", lastLiveNodes);
- state.put("nodeNameVsTimeAdded", nodeNameVsTimeAdded);
- return state;
- }
-
- @Override
- protected void setState(Map state) {
- this.lastLiveNodes.clear();
- this.nodeNameVsTimeAdded.clear();
- @SuppressWarnings({"unchecked"})
- Collection lastLiveNodes = (Collection)state.get("lastLiveNodes");
- if (lastLiveNodes != null) {
- this.lastLiveNodes.addAll(lastLiveNodes);
- }
- @SuppressWarnings({"unchecked"})
- Map nodeNameVsTimeAdded = (Map)state.get("nodeNameVsTimeAdded");
- if (nodeNameVsTimeAdded != null) {
- this.nodeNameVsTimeAdded.putAll(nodeNameVsTimeAdded);
- }
- }
-
- @Override
- public void run() {
- try {
- synchronized (this) {
- if (isClosed) {
- log.warn("NodeAddedTrigger ran but was already closed");
- throw new RuntimeException("Trigger has been closed");
- }
- }
- log.debug("Running NodeAddedTrigger {}", name);
-
- Set newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
- if (log.isDebugEnabled()) {
- log.debug("Found livenodes: {}", newLiveNodes.size());
- }
-
- // have any nodes that we were tracking been removed from the cluster?
- // if so, remove them from the tracking map
- Set trackingKeySet = nodeNameVsTimeAdded.keySet();
- trackingKeySet.retainAll(newLiveNodes);
-
- // have any new nodes been added?
- Set copyOfNew = new HashSet<>(newLiveNodes);
- copyOfNew.removeAll(lastLiveNodes);
- copyOfNew.forEach(n -> {
- long eventTime = cloudManager.getTimeSource().getTimeNs();
- log.debug("Tracking new node: {} at time {}", n, eventTime);
- nodeNameVsTimeAdded.put(n, eventTime);
- });
-
- // has enough time expired to trigger events for a node?
- List nodeNames = new ArrayList<>();
- List times = new ArrayList<>();
- for (Iterator> it = nodeNameVsTimeAdded.entrySet().iterator(); it.hasNext(); ) {
- Map.Entry entry = it.next();
- String nodeName = entry.getKey();
- Long timeAdded = entry.getValue();
- long now = cloudManager.getTimeSource().getTimeNs();
- if (TimeUnit.SECONDS.convert(now - timeAdded, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
- nodeNames.add(nodeName);
- times.add(timeAdded);
- }
- }
- AutoScaling.TriggerEventProcessor processor = processorRef.get();
- if (!nodeNames.isEmpty()) {
- if (processor != null) {
- if (log.isDebugEnabled()) {
- log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name,
- nodeNames, times, cloudManager.getTimeSource().getTimeNs());
- }
- if (processor.process(new NodeAddedEvent(getEventType(), getName(), times, nodeNames, preferredOp, replicaType))) {
- // remove from tracking set only if the fire was accepted
- nodeNames.forEach(n -> {
- log.debug("Removing new node from tracking: {}", n);
- nodeNameVsTimeAdded.remove(n);
- });
- } else {
- log.debug("Processor returned false for {}!", nodeNames);
- }
- } else {
- nodeNames.forEach(n -> {
- nodeNameVsTimeAdded.remove(n);
- });
- }
- }
- lastLiveNodes = new HashSet<>(newLiveNodes);
- } catch (RuntimeException e) {
- log.error("Unexpected exception in NodeAddedTrigger", e);
- }
- }
-
- public static class NodeAddedEvent extends TriggerEvent {
-
- public NodeAddedEvent(TriggerEventType eventType, String source, List times, List nodeNames, String preferredOp, Replica.Type replicaType) {
- // use the oldest time as the time of the event
- super(eventType, source, times.get(0), null);
- properties.put(NODE_NAMES, nodeNames);
- properties.put(EVENT_TIMES, times);
- properties.put(PREFERRED_OP, preferredOp);
- properties.put(REPLICA_TYPE, replicaType);
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
deleted file mode 100644
index b1c58183a39..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_ACTIVE;
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_INACTIVE;
-import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_STATE;
-import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
-
-/**
- * Trigger for the {@link TriggerEventType#NODELOST} event
- */
-public class NodeLostTrigger extends TriggerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private Set lastLiveNodes = new HashSet<>();
-
- private Map nodeNameVsTimeRemoved = new HashMap<>();
-
- private String preferredOp;
-
- public NodeLostTrigger(String name) {
- super(TriggerEventType.NODELOST, name);
- TriggerUtils.validProperties(validProperties, PREFERRED_OP);
- }
-
- @Override
- public void init() throws Exception {
- super.init();
- lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
- log.debug("NodeLostTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
- // pick up lost nodes for which marker paths were created
- try {
- List lost = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
- lost.forEach(n -> {
- String markerPath = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + n;
- try {
- Map markerData = Utils.getJson(stateManager, markerPath);
- // skip inactive markers
- if (markerData.getOrDefault(MARKER_STATE, MARKER_ACTIVE).equals(MARKER_INACTIVE)) {
- return;
- }
- } catch (InterruptedException | IOException | KeeperException e) {
- log.debug("-- ignoring marker {} state due to error", markerPath, e);
- }
- // don't add nodes that have since came back
- if (!lastLiveNodes.contains(n) && !nodeNameVsTimeRemoved.containsKey(n)) {
- // since {@code #restoreState(AutoScaling.Trigger)} is called first, the timeRemoved for a node may also be restored
- log.debug("Adding lost node from marker path: {}", n);
- nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTimeNs());
- }
- });
- } catch (NoSuchElementException e) {
- // ignore
- } catch (Exception e) {
- log.warn("Exception retrieving nodeLost markers", e);
- }
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
- preferredOp = preferredOp.toLowerCase(Locale.ROOT);
- // verify
- CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
- switch (action) {
- case MOVEREPLICA:
- case DELETENODE:
- case NONE:
- break;
- default:
- throw new TriggerValidationException("Unsupported preferredOperation=" + preferredOp + " specified for node lost trigger");
- }
- }
-
- @Override
- public void restoreState(AutoScaling.Trigger old) {
- assert old.isClosed();
- if (old instanceof NodeLostTrigger) {
- NodeLostTrigger that = (NodeLostTrigger) old;
- assert this.name.equals(that.name);
- this.lastLiveNodes.clear();
- this.lastLiveNodes.addAll(that.lastLiveNodes);
- this.nodeNameVsTimeRemoved.clear();
- this.nodeNameVsTimeRemoved.putAll(that.nodeNameVsTimeRemoved);
- } else {
- throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
- "Unable to restore state from an unknown type of trigger");
- }
- }
-
- @Override
- protected Map getState() {
- Map state = new HashMap<>();
- state.put("lastLiveNodes", lastLiveNodes);
- state.put("nodeNameVsTimeRemoved", nodeNameVsTimeRemoved);
- return state;
- }
-
- @Override
- protected void setState(Map state) {
- this.lastLiveNodes.clear();
- this.nodeNameVsTimeRemoved.clear();
- @SuppressWarnings({"unchecked"})
- Collection lastLiveNodes = (Collection)state.get("lastLiveNodes");
- if (lastLiveNodes != null) {
- this.lastLiveNodes.addAll(lastLiveNodes);
- }
- @SuppressWarnings({"unchecked"})
- Map nodeNameVsTimeRemoved = (Map)state.get("nodeNameVsTimeRemoved");
- if (nodeNameVsTimeRemoved != null) {
- this.nodeNameVsTimeRemoved.putAll(nodeNameVsTimeRemoved);
- }
- }
-
- @Override
- public void run() {
- try {
- synchronized (this) {
- if (isClosed) {
- log.warn("NodeLostTrigger ran but was already closed");
- return;
- }
- }
-
- Set newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
- if (log.isDebugEnabled()) {
- log.debug("Running NodeLostTrigger: {} with currently live nodes: {} and last live nodes: {}", name, newLiveNodes.size(), lastLiveNodes.size());
- }
- log.trace("Current Live Nodes for {}: {}", name, newLiveNodes);
- log.trace("Last Live Nodes for {}: {}", name, lastLiveNodes);
-
- // have any nodes that we were tracking been added to the cluster?
- // if so, remove them from the tracking map
- Set trackingKeySet = nodeNameVsTimeRemoved.keySet();
- trackingKeySet.removeAll(newLiveNodes);
-
- // have any nodes been removed?
- Set copyOfLastLiveNodes = new HashSet<>(lastLiveNodes);
- copyOfLastLiveNodes.removeAll(newLiveNodes);
- copyOfLastLiveNodes.forEach(n -> {
- log.debug("Tracking lost node: {}", n);
- nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTimeNs());
- });
-
- // has enough time expired to trigger events for a node?
- List nodeNames = new ArrayList<>();
- List times = new ArrayList<>();
- for (Iterator> it = nodeNameVsTimeRemoved.entrySet().iterator(); it.hasNext(); ) {
- Map.Entry entry = it.next();
- String nodeName = entry.getKey();
- Long timeRemoved = entry.getValue();
- long now = cloudManager.getTimeSource().getTimeNs();
- long te = TimeUnit.SECONDS.convert(now - timeRemoved, TimeUnit.NANOSECONDS);
- if (te >= getWaitForSecond()) {
- nodeNames.add(nodeName);
- times.add(timeRemoved);
- }
- }
- // fire!
- AutoScaling.TriggerEventProcessor processor = processorRef.get();
- if (!nodeNames.isEmpty()) {
- if (processor != null) {
- log.debug("NodeLostTrigger firing registered processor for lost nodes: {}", nodeNames);
- if (processor.process(new NodeLostEvent(getEventType(), getName(), times, nodeNames, preferredOp))) {
- // remove from tracking set only if the fire was accepted
- nodeNames.forEach(n -> {
- nodeNameVsTimeRemoved.remove(n);
- });
- } else {
- log.debug("NodeLostTrigger processor for lost nodes: {} is not ready, will try later", nodeNames);
- }
- } else {
- log.debug("NodeLostTrigger firing, but no processor - so removing lost nodes: {}", nodeNames);
- nodeNames.forEach(n -> {
- nodeNameVsTimeRemoved.remove(n);
- });
- }
- }
- lastLiveNodes = new HashSet<>(newLiveNodes);
- } catch (AlreadyClosedException e) {
-
- } catch (RuntimeException e) {
- log.error("Unexpected exception in NodeLostTrigger", e);
- }
- }
-
- public static class NodeLostEvent extends TriggerEvent {
-
- public NodeLostEvent(TriggerEventType eventType, String source, List times, List nodeNames, String preferredOp) {
- // use the oldest time as the time of the event
- super(eventType, source, times.get(0), null);
- properties.put(NODE_NAMES, nodeNames);
- properties.put(EVENT_TIMES, times);
- properties.put(PREFERRED_OP, preferredOp);
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
deleted file mode 100644
index fa27942bd9c..00000000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
+++ /dev/null
@@ -1,407 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.autoscaling;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.net.ConnectException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.solr.client.solrj.cloud.DistribStateManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.SolrCloseable;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-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.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
-
-/**
- * Overseer thread responsible for reading triggers from zookeeper and
- * adding/removing them from {@link ScheduledTriggers}
- */
-public class OverseerTriggerThread implements Runnable, SolrCloseable {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public static final String MARKER_STATE = "state";
- public static final String MARKER_ACTIVE = "active";
- public static final String MARKER_INACTIVE = "inactive";
- public static final int DEFAULT_AUTO_ADD_REPLICA_WAIT_FOR_SECONDS = 120;
-
-
- private final SolrCloudManager cloudManager;
-
- private final ScheduledTriggers scheduledTriggers;
-
- private final AutoScaling.TriggerFactory triggerFactory;
-
- private final ReentrantLock updateLock = new ReentrantLock();
-
- private final Condition updated = updateLock.newCondition();
-
- /*
- Following variables are only accessed or modified when updateLock is held
- */
- private int znodeVersion = 0;
-
- private Map activeTriggers = new HashMap<>();
-
- private volatile int processedZnodeVersion = -1;
-
- private volatile boolean isClosed = false;
-
- private AutoScalingConfig autoScalingConfig;
-
- public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager cloudManager) {
- this.cloudManager = cloudManager;
- scheduledTriggers = new ScheduledTriggers(loader, cloudManager);
- triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
- }
-
- @Override
- public void close() throws IOException {
- updateLock.lock();
- try {
- isClosed = true;
- activeTriggers.clear();
- updated.signalAll();
- } finally {
- updateLock.unlock();
- }
- IOUtils.closeQuietly(triggerFactory);
- IOUtils.closeQuietly(scheduledTriggers);
- log.debug("OverseerTriggerThread has been closed explicitly");
- }
-
- /**
- * For tests.
- * @lucene.internal
- * @return current {@link ScheduledTriggers} instance
- */
- public ScheduledTriggers getScheduledTriggers() {
- return scheduledTriggers;
- }
-
- /**
- * For tests, to ensure that all processing has been completed in response to an update of /autoscaling.json.
- * @lucene.internal
- * @return version of /autoscaling.json for which all configuration updates & processing have been completed.
- * Until then this value will always be smaller than the current znodeVersion of /autoscaling.json.
- */
- public int getProcessedZnodeVersion() {
- return processedZnodeVersion;
- }
-
- @Override
- public boolean isClosed() {
- return isClosed;
- }
-
- @Override
- public void run() {
- int lastZnodeVersion = znodeVersion;
-
- // we automatically add a trigger for auto add replicas if it does not exists already
- // we also automatically add a scheduled maintenance trigger
- while (!isClosed) {
- try {
- if (Thread.currentThread().isInterrupted()) {
- log.warn("Interrupted");
- break;
- }
- AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
- AutoScalingConfig updatedConfig = withDefaultPolicy(autoScalingConfig);
- updatedConfig = withAutoAddReplicasTrigger(updatedConfig);
- updatedConfig = withScheduledMaintenanceTrigger(updatedConfig);
- if (updatedConfig.equals(autoScalingConfig)) break;
- log.debug("Adding .auto_add_replicas and .scheduled_maintenance triggers");
- cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(updatedConfig), updatedConfig.getZkVersion());
- break;
- } catch (AlreadyClosedException e) {
- break;
- } catch (BadVersionException bve) {
- // somebody else has changed the configuration so we must retry
- } catch (InterruptedException e) {
- // Restore the interrupted status
- Thread.currentThread().interrupt();
- log.warn("Interrupted", e);
- break;
- }
- catch (IOException | KeeperException e) {
- if (e instanceof KeeperException.SessionExpiredException ||
- (e.getCause()!=null && e.getCause() instanceof KeeperException.SessionExpiredException)) {
- log.warn("Solr cannot talk to ZK, exiting {} main queue loop"
- , getClass().getSimpleName(), e);
- return;
- } else {
- log.error("A ZK error has occurred", e);
- }
- }
- }
-
- if (isClosed || Thread.currentThread().isInterrupted()) return;
-
- try {
- refreshAutoScalingConf(new AutoScalingWatcher());
- } catch (ConnectException e) {
- log.warn("ZooKeeper watch triggered for autoscaling conf, but Solr cannot talk to ZK: ", e);
- } catch (InterruptedException e) {
- // Restore the interrupted status
- Thread.currentThread().interrupt();
- log.warn("Interrupted", e);
- } catch (Exception e) {
- log.error("Unexpected exception", e);
- }
-
- while (true) {
- Map copy = null;
- try {
-
- updateLock.lockInterruptibly();
- try {
- // must check for close here before we await on the condition otherwise we can
- // only be woken up on interruption
- if (isClosed) {
- log.info("OverseerTriggerThread has been closed, exiting.");
- break;
- }
-
- log.debug("Current znodeVersion {}, lastZnodeVersion {}", znodeVersion, lastZnodeVersion);
-
- if (znodeVersion == lastZnodeVersion) {
- updated.await();
-
- // are we closed?
- if (isClosed) {
- log.info("OverseerTriggerThread woken up but we are closed, exiting.");
- break;
- }
-
- // spurious wakeup?
- if (znodeVersion == lastZnodeVersion) continue;
- }
- copy = new HashMap<>(activeTriggers);
- lastZnodeVersion = znodeVersion;
- log.debug("Processed trigger updates upto znodeVersion {}", znodeVersion);
- } finally {
- updateLock.unlock();
- }
- } catch (InterruptedException e) {
- // Restore the interrupted status
- Thread.currentThread().interrupt();
- log.warn("Interrupted", e);
- break;
- }
-
- // update the current config
- scheduledTriggers.setAutoScalingConfig(autoScalingConfig);
-
- Set managedTriggerNames = scheduledTriggers.getScheduledTriggerNames();
- // remove the triggers which are no longer active
- for (String managedTriggerName : managedTriggerNames) {
- if (!copy.containsKey(managedTriggerName)) {
- scheduledTriggers.remove(managedTriggerName);
- }
- }
- // nodeLost / nodeAdded markers are checked by triggers during their init() call
- // which is invoked in scheduledTriggers.add(), so once this is done we can remove them
- try {
- // add new triggers and/or replace and close the replaced triggers
- for (Map.Entry entry : copy.entrySet()) {
- try {
- scheduledTriggers.add(entry.getValue());
- } catch (AlreadyClosedException e) {
-
- } catch (Exception e) {
- log.warn("Exception initializing trigger {}, configuration ignored", entry.getKey(), e);
- }
- }
- } catch (AlreadyClosedException e) {
- // this _should_ mean that we're closing, complain loudly if that's not the case
- if (isClosed) {
- return;
- } else {
- throw new IllegalStateException("Caught AlreadyClosedException from ScheduledTriggers, but we're not closed yet!", e);
- }
- }
- log.debug("-- deactivating old nodeLost / nodeAdded markers");
- deactivateMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
- deactivateMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
- processedZnodeVersion = znodeVersion;
- }
- }
-
- private void deactivateMarkers(String path) {
- DistribStateManager stateManager = cloudManager.getDistribStateManager();
- try {
- List markers = stateManager.listData(path);
- for (String marker : markers) {
- String markerPath = path + "/" + marker;
- try {
- Map markerMap = new HashMap<>(Utils.getJson(stateManager, markerPath));
- markerMap.put(MARKER_STATE, MARKER_INACTIVE);
- stateManager.setData(markerPath, Utils.toJSON(markerMap), -1);
- } catch (NoSuchElementException e) {
- // ignore - already deleted
- }
- }
- } catch (NoSuchElementException e) {
- // ignore
- } catch (Exception e) {
- log.warn("Error deactivating old markers", e);
- }
- }
-
- class AutoScalingWatcher implements Watcher {
- @Override
- public void process(WatchedEvent watchedEvent) {
- // session events are not change events, and do not remove the watcher
- if (Event.EventType.None.equals(watchedEvent.getType())) {
- return;
- }
-
- try {
- refreshAutoScalingConf(this);
- } catch (ConnectException e) {
- log.warn("ZooKeeper watch triggered for autoscaling conf, but we cannot talk to ZK: ", e);
- } catch (InterruptedException e) {
- // Restore the interrupted status
- Thread.currentThread().interrupt();
- log.warn("Interrupted", e);
- } catch (Exception e) {
- log.error("Unexpected exception", e);
- }
- }
-
- }
-
- private void refreshAutoScalingConf(Watcher watcher) throws InterruptedException, IOException {
- updateLock.lock();
- try {
- if (isClosed) {
- return;
- }
- AutoScalingConfig currentConfig = cloudManager.getDistribStateManager().getAutoScalingConfig(watcher);
- if (log.isDebugEnabled()) {
- log.debug("Refreshing {} with znode version {}", ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, currentConfig.getZkVersion());
- }
- if (znodeVersion >= currentConfig.getZkVersion()) {
- // protect against reordered watcher fires by ensuring that we only move forward
- return;
- }
- autoScalingConfig = currentConfig;
- znodeVersion = autoScalingConfig.getZkVersion();
- Map triggerMap = loadTriggers(triggerFactory, autoScalingConfig);
-
- // remove all active triggers that have been removed from ZK
- Set trackingKeySet = activeTriggers.keySet();
- trackingKeySet.retainAll(triggerMap.keySet());
-
- // now lets add or remove triggers which have been enabled or disabled respectively
- for (Map.Entry