You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2020/07/20 12:19:16 UTC
[lucene-solr] branch master updated: SOLR-14656: Removing
Autoscaling Framework
This is an automated email from the ASF dual-hosted git repository.
ishan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
The following commit(s) were added to refs/heads/master by this push:
new cc0c111 SOLR-14656: Removing Autoscaling Framework
cc0c111 is described below
commit cc0c111949d5039a0c7cb67cad55c63e2f761298
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Mon Jul 20 17:47:01 2020 +0530
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 +-
.../src/java/org/apache/solr/cloud/CloudUtil.java | 1 -
.../src/java/org/apache/solr/cloud/Overseer.java | 13 +-
.../java/org/apache/solr/cloud/ZkController.java | 41 -
.../solr/cloud/api/collections/AddReplicaCmd.java | 17 +-
.../apache/solr/cloud/api/collections/Assign.java | 139 +-
.../cloud/api/collections/CreateCollectionCmd.java | 20 +-
.../OverseerCollectionMessageHandler.java | 8 +-
.../api/collections/ReindexCollectionCmd.java | 5 +-
.../solr/cloud/api/collections/ReplaceNodeCmd.java | 7 -
.../solr/cloud/api/collections/RestoreCmd.java | 364 +-
.../solr/cloud/api/collections/SplitShardCmd.java | 52 +-
.../solr/cloud/api/collections/UtilizeNodeCmd.java | 139 -
.../solr/cloud/autoscaling/ActionContext.java | 68 -
.../autoscaling/AutoAddReplicasPlanAction.java | 40 -
.../apache/solr/cloud/autoscaling/AutoScaling.java | 246 -
.../solr/cloud/autoscaling/AutoScalingHandler.java | 742 --
.../solr/cloud/autoscaling/ComputePlanAction.java | 355 -
.../solr/cloud/autoscaling/ExecutePlanAction.java | 250 -
.../cloud/autoscaling/HttpTriggerListener.java | 164 -
.../autoscaling/InactiveMarkersPlanAction.java | 141 -
.../cloud/autoscaling/InactiveShardPlanAction.java | 163 -
.../solr/cloud/autoscaling/IndexSizeTrigger.java | 562 -
.../solr/cloud/autoscaling/LoggingListener.java | 38 -
.../solr/cloud/autoscaling/MetricTrigger.java | 220 -
.../solr/cloud/autoscaling/NodeAddedTrigger.java | 250 -
.../solr/cloud/autoscaling/NodeLostTrigger.java | 240 -
.../cloud/autoscaling/OverseerTriggerThread.java | 407 -
.../solr/cloud/autoscaling/ScheduledTrigger.java | 227 -
.../solr/cloud/autoscaling/ScheduledTriggers.java | 913 --
.../solr/cloud/autoscaling/SearchRateTrigger.java | 805 --
.../solr/cloud/autoscaling/SystemLogListener.java | 221 -
.../solr/cloud/autoscaling/TriggerAction.java | 51 -
.../solr/cloud/autoscaling/TriggerActionBase.java | 87 -
.../cloud/autoscaling/TriggerActionException.java | 33 -
.../apache/solr/cloud/autoscaling/TriggerBase.java | 300 -
.../solr/cloud/autoscaling/TriggerEvent.java | 315 -
.../solr/cloud/autoscaling/TriggerEventQueue.java | 121 -
.../solr/cloud/autoscaling/TriggerListener.java | 65 -
.../cloud/autoscaling/TriggerListenerBase.java | 97 -
.../solr/cloud/autoscaling/TriggerUtils.java | 88 -
.../autoscaling/TriggerValidationException.java | 74 -
.../solr/cloud/autoscaling/package-info.java | 21 -
.../solr/cloud/autoscaling/sim/ActionError.java | 24 -
.../cloud/autoscaling/sim/FakeDocIterator.java | 56 -
.../autoscaling/sim/GenericDistributedQueue.java | 601 -
.../sim/GenericDistributedQueueFactory.java | 45 -
.../solr/cloud/autoscaling/sim/LiveNodesSet.java | 113 -
.../sim/NoopDistributedQueueFactory.java | 94 -
.../cloud/autoscaling/sim/SimCloudManager.java | 1060 --
.../autoscaling/sim/SimClusterStateProvider.java | 2574 ----
.../autoscaling/sim/SimDistribStateManager.java | 649 -
.../sim/SimDistributedQueueFactory.java | 287 -
.../autoscaling/sim/SimNodeStateProvider.java | 359 -
.../solr/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 -
.../autoscaling/sim/SnapshotNodeStateProvider.java | 203 -
.../solr/cloud/autoscaling/sim/package-info.java | 94 -
.../apache/solr/cloud/overseer/ReplicaMutator.java | 2 +-
.../java/org/apache/solr/core/CoreContainer.java | 12 -
.../org/apache/solr/core/SolrResourceLoader.java | 3 +-
.../handler/admin/AutoscalingHistoryHandler.java | 166 -
.../solr/handler/admin/CollectionsHandler.java | 12 +-
.../solr/handler/admin/MetricsHistoryHandler.java | 6 +-
.../solr/security/PermissionNameProvider.java | 3 -
.../src/java/org/apache/solr/util/SolrCLI.java | 390 -
.../solr/simSnapshot/autoscalingState.json | 3923 ------
.../test-files/solr/simSnapshot/clusterState.json | 2838 ----
.../test-files/solr/simSnapshot/distribState.json | 206 -
.../test-files/solr/simSnapshot/managerState.json | 1 -
.../src/test-files/solr/simSnapshot/nodeState.json | 3823 ------
.../test-files/solr/simSnapshot/statistics.json | 2029 ---
.../test/org/apache/solr/cloud/CloudTestUtils.java | 195 -
.../apache/solr/cloud/CollectionsAPISolrJTest.java | 18 +-
.../solr/cloud/MetricsHistoryIntegrationTest.java | 24 +-
.../org/apache/solr/cloud/MoveReplicaTest.java | 2 -
.../OverseerCollectionConfigSetProcessorTest.java | 11 +-
.../apache/solr/cloud/ReplaceNodeNoTargetTest.java | 127 -
.../cloud/RoutingToNodesWithPropertiesTest.java | 242 -
.../cloud/SharedFSAutoReplicaFailoverTest.java | 14 +-
.../apache/solr/cloud/TestClusterProperties.java | 12 -
.../org/apache/solr/cloud/TestUtilizeNode.java | 184 -
.../org/apache/solr/cloud/TestWithCollection.java | 611 -
.../AbstractCloudBackupRestoreTestCase.java | 4 -
.../solr/cloud/api/collections/AssignTest.java | 204 -
.../collections/CollectionTooManyReplicasTest.java | 13 +-
.../ConcurrentCreateCollectionTest.java | 292 -
.../AutoAddReplicasIntegrationTest.java | 472 -
.../autoscaling/AutoAddReplicasPlanActionTest.java | 263 -
.../cloud/autoscaling/AutoScalingHandlerTest.java | 1114 --
.../solr/cloud/autoscaling/CapturedEvent.java | 65 -
.../cloud/autoscaling/ComputePlanActionTest.java | 794 --
.../cloud/autoscaling/ExecutePlanActionTest.java | 381 -
.../HdfsAutoAddReplicasIntegrationTest.java | 61 -
.../cloud/autoscaling/HttpTriggerListenerTest.java | 208 -
.../IndexSizeTriggerMixedBoundsTest.java | 374 -
.../IndexSizeTriggerSizeEstimationTest.java | 323 -
.../cloud/autoscaling/IndexSizeTriggerTest.java | 795 --
.../autoscaling/MetricTriggerIntegrationTest.java | 242 -
.../solr/cloud/autoscaling/MetricTriggerTest.java | 135 -
.../NodeAddedTriggerIntegrationTest.java | 320 -
.../cloud/autoscaling/NodeAddedTriggerTest.java | 345 -
.../NodeLostTriggerIntegrationTest.java | 356 -
.../cloud/autoscaling/NodeLostTriggerTest.java | 394 -
.../autoscaling/NodeMarkersRegistrationTest.java | 356 -
.../cloud/autoscaling/RestoreTriggerStateTest.java | 168 -
.../ScheduledMaintenanceTriggerTest.java | 384 -
.../ScheduledTriggerIntegrationTest.java | 150 -
.../cloud/autoscaling/ScheduledTriggerTest.java | 141 -
.../SearchRateTriggerIntegrationTest.java | 747 --
.../cloud/autoscaling/SearchRateTriggerTest.java | 357 -
.../cloud/autoscaling/SystemLogListenerTest.java | 293 -
.../solr/cloud/autoscaling/TestPolicyCloud.java | 542 -
.../TriggerCooldownIntegrationTest.java | 228 -
.../cloud/autoscaling/TriggerEventQueueTest.java | 99 -
.../cloud/autoscaling/TriggerIntegrationTest.java | 729 -
.../TriggerSetPropertiesIntegrationTest.java | 275 -
.../autoscaling/sim/SimSolrCloudTestCase.java | 255 -
.../sim/TestSimClusterStateProvider.java | 227 -
.../autoscaling/sim/TestSimComputePlanAction.java | 401 -
.../sim/TestSimDistribStateManager.java | 380 -
.../autoscaling/sim/TestSimDistributedQueue.java | 221 -
.../autoscaling/sim/TestSimExecutePlanAction.java | 224 -
.../autoscaling/sim/TestSimExtremeIndexing.java | 147 -
.../sim/TestSimGenericDistributedQueue.java | 38 -
.../cloud/autoscaling/sim/TestSimLargeCluster.java | 870 --
.../autoscaling/sim/TestSimNodeAddedTrigger.java | 331 -
.../autoscaling/sim/TestSimNodeLostTrigger.java | 350 -
.../cloud/autoscaling/sim/TestSimPolicyCloud.java | 379 -
.../cloud/autoscaling/sim/TestSimScenario.java | 173 -
.../autoscaling/sim/TestSimTriggerIntegration.java | 1551 ---
.../solr/cloud/autoscaling/sim/TestSimUtils.java | 95 -
.../autoscaling/sim/TestSnapshotCloudManager.java | 275 -
.../org/apache/solr/cloud/rule/RuleEngineTest.java | 2 +-
.../test/org/apache/solr/cloud/rule/RulesTest.java | 67 -
.../apache/solr/handler/V2ApiIntegrationTest.java | 4 +-
.../admin/AutoscalingHistoryHandlerTest.java | 463 -
.../handler/admin/MetricsHistoryHandlerTest.java | 43 +-
.../solr/handler/admin/TestCollectionAPIs.java | 4 +-
.../security/JWTAuthPluginIntegrationTest.java | 6 +-
.../hadoop/TestSolrCloudWithHadoopAuthPlugin.java | 6 +-
.../apache/solr/util/TestSolrCLIRunExample.java | 117 -
.../src/test/org/apache/solr/util/TestUtils.java | 1 -
.../src/cluster-node-management.adoc | 26 +-
solr/solr-ref-guide/src/collection-management.adoc | 19 +-
.../solr-ref-guide/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 +-
.../solr-ref-guide/src/migrate-to-policy-rule.adoc | 198 -
solr/solr-ref-guide/src/replica-management.adoc | 2 -
.../src/rule-based-authorization-plugin.adoc | 15 -
solr/solr-ref-guide/src/running-solr-on-hdfs.adoc | 39 -
solr/solr-ref-guide/src/shard-management.adoc | 6 +-
solr/solr-ref-guide/src/solr-upgrade-notes.adoc | 4 +-
.../src/solrcloud-autoscaling-api.adoc | 808 --
.../solrcloud-autoscaling-auto-add-replicas.adoc | 73 -
.../src/solrcloud-autoscaling-fault-tolerance.adoc | 59 -
.../src/solrcloud-autoscaling-listeners.adoc | 220 -
.../src/solrcloud-autoscaling-overview.adoc | 116 -
.../solrcloud-autoscaling-policy-preferences.adoc | 638 -
.../src/solrcloud-autoscaling-trigger-actions.adoc | 145 -
.../src/solrcloud-autoscaling-triggers.adoc | 627 -
solr/solr-ref-guide/src/solrcloud-autoscaling.adoc | 35 -
solr/solr-ref-guide/src/solrcloud.adoc | 6 +-
solr/solr-ref-guide/src/suggestions-screen.adoc | 40 -
...ing-the-solr-administration-user-interface.adoc | 3 +-
solr/solr-ref-guide/src/v2-api.adoc | 7 +-
.../{autoscaling => }/AlreadyExistsException.java | 2 +-
.../{autoscaling => }/BadVersionException.java | 2 +-
.../client/solrj/cloud/DelegatingCloudManager.java | 89 +
.../cloud/DelegatingClusterStateProvider.java | 130 +
.../client/solrj/cloud/DistribStateManager.java | 11 -
.../cloud/{autoscaling => }/NotEmptyException.java | 2 +-
.../cloud/{autoscaling => }/VersionedData.java | 2 +-
.../cloud/autoscaling/AddReplicaSuggester.java | 84 -
.../solrj/cloud/autoscaling/AutoScalingConfig.java | 616 -
.../solr/client/solrj/cloud/autoscaling/Cell.java | 74 -
.../client/solrj/cloud/autoscaling/Clause.java | 839 --
.../solrj/cloud/autoscaling/ComputedType.java | 99 -
.../client/solrj/cloud/autoscaling/Condition.java | 125 -
.../solrj/cloud/autoscaling/CoresVariable.java | 127 -
.../cloud/autoscaling/DelegatingCloudManager.java | 93 -
.../DelegatingClusterStateProvider.java | 130 -
.../autoscaling/DelegatingDistribStateManager.java | 107 -
.../autoscaling/DelegatingNodeStateProvider.java | 57 -
.../cloud/autoscaling/DeleteNodeSuggester.java | 48 -
.../cloud/autoscaling/DeleteReplicaSuggester.java | 78 -
.../solrj/cloud/autoscaling/FreeDiskVariable.java | 176 -
.../cloud/autoscaling/MoveReplicaSuggester.java | 115 -
.../solrj/cloud/autoscaling/NodeVariable.java | 44 -
.../solrj/cloud/autoscaling/NoneSuggester.java | 41 -
.../client/solrj/cloud/autoscaling/Operand.java | 210 -
.../client/solrj/cloud/autoscaling/Policy.java | 792 --
.../solrj/cloud/autoscaling/PolicyHelper.java | 733 --
.../client/solrj/cloud/autoscaling/Preference.java | 157 -
.../client/solrj/cloud/autoscaling/RangeVal.java | 61 -
.../solrj/cloud/autoscaling/ReplicaCount.java | 134 -
.../solrj/cloud/autoscaling/ReplicaVariable.java | 176 -
.../solr/client/solrj/cloud/autoscaling/Row.java | 402 -
.../solrj/cloud/autoscaling/SealedClause.java | 29 -
.../cloud/autoscaling/SplitShardSuggester.java | 68 -
.../client/solrj/cloud/autoscaling/Suggester.java | 530 -
.../client/solrj/cloud/autoscaling/Suggestion.java | 133 -
.../autoscaling/TriggerEventProcessorStage.java | 30 -
.../solrj/cloud/autoscaling/TriggerEventType.java | 33 -
.../cloud/autoscaling/UnsupportedSuggester.java | 61 -
.../client/solrj/cloud/autoscaling/Variable.java | 410 -
.../solrj/cloud/autoscaling/VariableBase.java | 213 -
.../client/solrj/cloud/autoscaling/Violation.java | 182 -
.../cloud/autoscaling/WithCollectionVariable.java | 170 -
.../solrj/cloud/autoscaling/package-info.java | 23 -
.../solrj/impl/SolrClientNodeStateProvider.java | 85 +-
.../client/solrj/impl/ZkDistribStateManager.java | 34 +-
.../solrj/request/CollectionAdminRequest.java | 42 -
.../client/solrj/request/CollectionApiMapping.java | 5 -
.../apache/solr/common/cloud/DocCollection.java | 19 -
.../apache/solr/common/cloud/ZkStateReader.java | 44 -
.../solr/common/params/AutoScalingParams.java | 76 -
.../solr/common/params/CollectionAdminParams.java | 6 -
.../solr/common/params/CollectionParams.java | 1 -
.../apache/solr/common/params/CommonParams.java | 10 +-
.../java/org/apache/solr/common/util/Utils.java | 2 +-
.../resources/apispec/autoscaling.Commands.json | 199 -
.../src/resources/apispec/autoscaling.history.json | 61 -
.../src/resources/apispec/cluster.Commands.json | 25 -
.../resources/apispec/collections.Commands.json | 11 -
.../collections.collection.Commands.modify.json | 11 -
.../solr/autoscaling/testAddMissingReplica.json | 122 -
.../solr/autoscaling/testAddTooManyPerPolicy.json | 129 -
.../autoscaling/testAutoScalingHandlerFailure.json | 141 -
...testAutoscalingPreferencesUsedWithNoPolicy.json | 52 -
.../autoscaling/testComputePlanAfterNodeAdded.json | 16 -
.../solr/autoscaling/testCoresSuggestions.json | 17 -
.../testCreateCollectionWithEmptyPolicy.json | 20 -
.../solrj/solr/autoscaling/testDiskSpaceHint.json | 16 -
.../solr/autoscaling/testEmptyCollection.json | 26 -
.../solrj/solr/autoscaling/testEqualOnNonNode.json | 83 -
.../solr/autoscaling/testFreeDiskDeviation.json | 35 -
.../solr/autoscaling/testFreeDiskSuggestions.json | 27 -
.../solr/autoscaling/testFreediskPercentage.json | 25 -
.../solrj/solr/autoscaling/testHostAttribute.json | 119 -
.../solrj/solr/autoscaling/testInfiniteLoop.json | 13173 -------------------
.../solr/autoscaling/testMoveReplicaSuggester.json | 15 -
.../testMoveReplicasInMultipleCollections.json | 86 -
.../solrj/solr/autoscaling/testPolicy.json | 41 -
.../solr/autoscaling/testPortSuggestions.json | 22 -
.../autoscaling/testReplicaCountSuggestions.json | 15 -
.../solr/autoscaling/testReplicaPercentage.json | 46 -
.../autoscaling/testReplicaZonesPercentage.json | 15 -
.../autoscaling/testScheduledTriggerFailure.json | 52 -
.../solrj/solr/autoscaling/testSortError.json | 225 -
.../autoscaling/testSuggestionsRebalance2.json | 131 -
.../autoscaling/testSuggestionsRebalanceOnly.json | 105 -
.../solr/autoscaling/testSysPropSuggestions.json | 127 -
.../solr/autoscaling/testSyspropSuggestions1.json | 24 -
.../solr/autoscaling/testUnresolvedSuggestion.json | 211 -
.../solr/autoscaling/testUtilizeNodeFailure.json | 69 -
.../solr/autoscaling/testUtilizeNodeFailure2.json | 66 -
.../solr/autoscaling/testViolationOutput.json | 22 -
.../solrj/solr/autoscaling/testWithCollection.json | 21 -
.../autoscaling/testWithCollectionMoveReplica.json | 28 -
.../testWithCollectionMoveVsAddSuggestions.json | 49 -
.../autoscaling/testWithCollectionSuggestions.json | 21 -
.../solrj/cloud/autoscaling/ConditionTest.java | 80 -
.../autoscaling/MoveReplicaSuggesterTest.java | 104 -
.../client/solrj/cloud/autoscaling/TestPolicy.java | 3358 -----
.../solrj/cloud/autoscaling/TestPolicy2.java | 540 -
.../solrj/cloud/autoscaling/TestPolicy2Old.java | 26 -
.../solrj/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 -
solr/webapp/web/js/angular/controllers/cloud.js | 2 -
.../js/angular/controllers/cluster-suggestions.js | 62 -
.../web/js/angular/controllers/collections.js | 6 +-
solr/webapp/web/partials/cluster_suggestions.html | 49 -
solr/webapp/web/partials/collection_overview.html | 3 -
solr/webapp/web/partials/collections.html | 12 -
287 files changed, 583 insertions(+), 80200 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 53ca9e0..ee5afa1 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 83d247c..7e3cf0c6 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 4b4d813..8fb5e7a 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 ecc653b..65de4ca 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.
* <p>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.</p>
* @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 3dfe6c5..bb405ad 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.</p>
*
* <p>The <b>Overseer</b> 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.</p>
+ * ZooKeeper that require global synchronization. </p>
*
* <p>The Overseer deals with:</p>
* <ul>
* <li>Cluster State updates, i.e. updating Collections' <code>state.json</code> files in ZooKeeper, see {@link ClusterStateUpdater},</li>
- * <li>Collection API implementation, including Autoscaling replica placement computation, see
+ * <li>Collection API implementation, see
* {@link OverseerCollectionConfigSetProcessor} and {@link OverseerCollectionMessageHandler} (and the example below),</li>
* <li>Updating Config Sets, see {@link OverseerCollectionConfigSetProcessor} and {@link OverseerConfigSetMessageHandler},</li>
- * <li>Autoscaling triggers, see {@link org.apache.solr.cloud.autoscaling.OverseerTriggerThread}.</li>
* </ul>
*
* <p>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<String, Object>() {
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 c50581c..ca44ece 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<Op> 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<Op> 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 1c2146b..ca5639f 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<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
- List<CreateReplica> createReplicas;
- try {
- createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount, sessionWrapper)
+ List<CreateReplica> 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<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
String collectionName, ZkNodeProps message,
- EnumMap<Replica.Type, Integer> replicaTypeVsCount,
- AtomicReference< PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+ EnumMap<Replica.Type, Integer> 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 98a399a..909b3ed 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;
}
- /**
- * <b>Note:</b> 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<DocCollection> collection, SolrCloudManager cloudManager) throws IOException, InterruptedException {
- boolean useLegacyAssignment = true;
- Map<String, Object> clusterProperties = cloudManager.getClusterStateProvider().getClusterProperties();
- if (clusterProperties.containsKey(CollectionAdminParams.DEFAULTS)) {
- Map<String, Object> defaults = (Map<String, Object>) clusterProperties.get(CollectionAdminParams.DEFAULTS);
- Map<String, Object> collectionDefaults = (Map<String, Object>) 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<ReplicaPosition> getPositionsUsingPolicy(String collName, List<String> shardNames,
- int nrtReplicas,
- int tlogReplicas,
- int pullReplicas,
- String policyName, SolrCloudManager cloudManager,
- List<String> nodesList) throws IOException, InterruptedException, AssignmentException {
- log.debug("shardnames {} NRT {} TLOG {} PULL {} , policy {}, nodeList {}", shardNames, nrtReplicas, tlogReplicas, pullReplicas, policyName, nodesList);
- List<ReplicaPosition> replicaPositions = null;
- AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
- try {
- Map<String, String> 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<String, ReplicaCount> getNodeNameVsShardCount(String collectionName,
ClusterState clusterState, List<String> createNodeList) {
HashMap<String, ReplicaCount> 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<ReplicaPosition> 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<Map> ruleMaps = (List<Map>) 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<Rule> 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 c3e9a3e..cfad397 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<String> shardNames = populateShardNames(message, router);
checkReplicaTypes(message);
- AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+
try {
@@ -187,7 +185,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
List<ReplicaPosition> 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<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
DocCollection docCollection,
ZkNodeProps message,
- List<String> shardNames,
- AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException, Assign.AssignmentException {
+ List<String> 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 e1fca93..74112b4 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 d98d50a..c45c772 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 f1c1f8c..aa10bb1e 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<PolicyHelper.SessionWrapper> 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 f800979..552c40a 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<String> 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<String, Slice> 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<String> 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<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
- sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
-
- 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);
- }
- HashMap<String, Object> 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;
- }
- }
+ 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<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
+
+ 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);
+ }
+ HashMap<String, Object> 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");
+ }
- // add async param
- if (asyncId != null) {
- propMap.put(ASYNC, asyncId);
+ // 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;
}
- 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.");
+ // add async param
+ if (asyncId != null) {
+ propMap.put(ASYNC, asyncId);
}
- 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<Object>());
- 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);
+ 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"));
}
- shardRequestTracker.processResponses(new NamedList(), shardHandler, true, "Could not restore core");
- }
+ countDownLatch.countDown();
+ });
+ }
- {
- ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
+ 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<Object>());
+ return;
+ }
- for (Slice s : restoreCollection.getSlices()) {
- for (Replica r : s.getReplicas()) {
- String nodeName = r.getNodeName();
- String coreNodeName = r.getCoreName();
- Replica.State stateRep = r.getState();
+ //refresh the location copy of collection state
+ restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
- if (log.isDebugEnabled()) {
- log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}", nodeName, coreNodeName,
- stateRep.name());
- }
+ {
+ 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");
+ }
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
- params.set(CoreAdminParams.NAME, coreNodeName);
+ {
+ ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
- shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
+ 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());
}
- shardRequestTracker.processResponses(new NamedList(), shardHandler, true,
- "REQUESTAPPLYUPDATES calls did not succeed");
+ 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<String, Object> 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))));
+ //Mark all shards in ACTIVE STATE
+ {
+ HashMap<String, Object> 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()) {
+ 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;
- //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++;
+ }
- // We already created either a NRT or an TLOG replica as leader
- if (numNrtReplicas > 0) {
+ for (int i = 1; i < totalReplicasPerShard; i++) {
+ Replica.Type typeToCreate;
+ if (createdNrtReplicas < numNrtReplicas) {
createdNrtReplicas++;
- } else if (createdTlogReplicas > 0) {
+ 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";
}
- 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<String, Object> 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);
+ if (log.isDebugEnabled()) {
+ log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
+ }
+ HashMap<String, Object> 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;
}
- ocmh.addPropertyParams(message, propMap);
+ }
- ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
+ // add async param
+ if (asyncId != null) {
+ propMap.put(ASYNC, asyncId);
}
- }
- }
+ ocmh.addPropertyParams(message, propMap);
- if (backupCollectionAlias != null && !backupCollectionAlias.equals(backupCollection)) {
- log.debug("Restoring alias {} -> {}", backupCollectionAlias, backupCollection);
- ocmh.zkStateReader.aliasesManager
- .applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(backupCollectionAlias, backupCollection));
+ ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
+ }
}
+ }
- log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
- } finally {
- if (sessionWrapper != null) sessionWrapper.release();
+ 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 0df9d09..495bf65 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<String> nodes = clusterState.getLiveNodes();
List<String> 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<ReplicaPosition> 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<String, Object> nodeValues = nodeStateProvider.getNodeValues(parentShardLeader.getNodeName(),
Collections.singletonList(ImplicitSnitch.DISK));
Map<String, Map<String, List<Replica>>> 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 0fd19ac..0000000
--- 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<ZkNodeProps> requests = new ArrayList<>();
- //first look for suggestions if any
- List<Suggester.SuggestionInfo> suggestions = PolicyHelper.getSuggestions(autoScalingConfig, ocmh.overseer.getSolrCloudManager());
- for (Suggester.SuggestionInfo suggestionInfo : suggestions) {
- if (log.isInfoEnabled()) {
- log.info("op: {}", suggestionInfo.getOperation());
- }
- String coll = null;
- List<String> 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<ZkNodeProps> 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 8487d3d..0000000
--- 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<String, Object> properties;
-
- public ActionContext(SolrCloudManager cloudManager, AutoScaling.Trigger source, Map<String, Object> properties) {
- this.cloudManager = cloudManager;
- this.source = source;
- this.properties = properties;
- }
-
- public SolrCloudManager getCloudManager() {
- return cloudManager;
- }
-
- public AutoScaling.Trigger getSource() {
- return source;
- }
-
- public Map<String, Object> 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<String, Object> 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 d129fdb..0000000
--- 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<String, Object> 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 1a191ee..0000000
--- 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.
- * <p>
- * 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.
- * <p>
- * 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<String, Object> getProperties();
-
- /** Number of seconds to wait between fired events ("waitFor" property). */
- int getWaitForSecond();
-
- /** Actions to execute when event is fired. */
- List<TriggerAction> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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 23ec075..0000000
--- 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<Map<String, String>> DEFAULT_ACTIONS = new ArrayList<>(3);
- private static Set<String> 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<String, String> 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<BiConsumer<SolrQueryResponse, AutoScalingConfig>> getSubpathExecutor(List<String> 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<String> 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<String> 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<CommandOperation> 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<CommandOperation> 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<String, Object> map = op.getDataMap() == null ? Collections.emptyMap() : op.getDataMap();
- Map<String, Object> 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<Map<String, Object>> clusterPolicy = (List<Map<String, Object>>) op.getCommandData();
- if (clusterPolicy == null || !(clusterPolicy instanceof List)) {
- op.addError("set-cluster-policy expects an array of objects");
- return currentConfig;
- }
- List<Clause> 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<Map<String, Object>> preferences = (List<Map<String, Object>>) op.getCommandData();
- if (preferences == null || !(preferences instanceof List)) {
- op.addError("A list of cluster preferences not found");
- return currentConfig;
- }
- List<Preference> 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<String, List<Clause>> 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<String, Object> policiesMap = op.getDataMap();
- for (Map.Entry<String, Object> 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<String, List<Clause>> currentClauses = new HashMap<>(currentConfig.getPolicy().getPolicies());
- Map<String, List<Clause>> newClauses = null;
- try {
- newClauses = Policy.clausesFromMap((Map<String, List<Map<String, Object>>>) 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<String, AutoScalingConfig.TriggerConfig> triggers = currentConfig.getTriggerConfigs();
- Set<String> changed = new HashSet<>();
- if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
- op.addError("No trigger exists with name: " + triggerName);
- return currentConfig;
- }
- Map<String, AutoScalingConfig.TriggerConfig> newTriggers = new HashMap<>();
- for (Map.Entry<String, AutoScalingConfig.TriggerConfig> 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<String, AutoScalingConfig.TriggerConfig> triggers = currentConfig.getTriggerConfigs();
- Set<String> changed = new HashSet<>();
-
- if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
- op.addError("No trigger exists with name: " + triggerName);
- return currentConfig;
- }
- Map<String, AutoScalingConfig.TriggerConfig> newTriggers = new HashMap<>();
- for (Map.Entry<String, AutoScalingConfig.TriggerConfig> 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<String, AutoScalingConfig.TriggerListenerConfig> 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<String> stageNames = op.getStrs(STAGE, Collections.emptyList());
- String listenerClass = op.getStr(CLASS);
- List<String> beforeActions = op.getStrs(BEFORE_ACTION, Collections.emptyList());
- List<String> afterActions = op.getStrs(AFTER_ACTION, Collections.emptyList());
-
- if (op.hasError()) return currentConfig;
-
- Map<String, AutoScalingConfig.TriggerConfig> 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<String> 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<Map<String, String>> actions = (List<Map<String, String>>) 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<String, String> 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<String, AutoScalingConfig.TriggerListenerConfig> 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<String, Object> 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<String, AutoScalingConfig.TriggerConfig> triggerConfigs = currentConfig.getTriggerConfigs();
- if (!triggerConfigs.containsKey(triggerName)) {
- op.addError("No trigger exists with name: " + triggerName);
- return currentConfig;
- }
- triggerConfigs = new HashMap<>(triggerConfigs);
- Set<String> activeListeners = new HashSet<>();
- Map<String, AutoScalingConfig.TriggerListenerConfig> 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<Api> 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 33bf6b0..0000000
--- 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.
- * <p>
- * 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<String> collectionsPredicate = s -> true;
-
- public ComputePlanAction() {
- super();
- TriggerUtils.validProperties(validProperties, "collections");
- }
-
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> 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<String> whiteListedCollections = StrUtils.splitSmart(colString, ',');
- collectionsPredicate = whiteListedCollections::contains;
- }
- } else if (value instanceof Map) {
- @SuppressWarnings({"unchecked"})
- Map<String, String> matchConditions = (Map<String, String>) 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<String, String> 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<String, Object> props = context.getProperties();
- props.compute("operations", (k, v) -> {
- @SuppressWarnings({"unchecked", "rawtypes"})
- List<SolrRequest> operations = (List<SolrRequest>) 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<TriggerEvent.Op> ops = (Collection<TriggerEvent.Op>) 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<TriggerEvent.Op> ops = (List<TriggerEvent.Op>)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<TriggerEvent.Op> unsupportedOps = (List<TriggerEvent.Op>)context.getProperties().computeIfAbsent(TriggerEvent.UNSUPPORTED_OPS, k -> new ArrayList<TriggerEvent.Op>());
- unsupportedOps.add(op);
- }
- for (Map.Entry<Suggester.Hint, Object> 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<String> srcNodes = (List<String>) 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<String> 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<Pair<String, String>> 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 1dfc3b1..0000000
--- 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<String, Object> 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<SolrRequest> operations = (List<SolrRequest>) 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<Object> result = response.getResponse();
- context.getProperties().compute("responses", (s, o) -> {
- @SuppressWarnings({"unchecked"})
- List<NamedList<Object>> responses = (List<NamedList<Object>>) 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 139efe0..0000000
--- 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:
- * <ul>
- * <li>config.* - listener configuration</li>
- * <li>event.* - event properties</li>
- * <li>stage - current stage of event processing</li>
- * <li>actionName - optional current action name</li>
- * <li>context.* - optional {@link ActionContext} properties</li>
- * <li>error - optional error string (from {@link Throwable#toString()})</li>
- * <li>message - optional message</li>
- * </ul>
- * The following listener configuration is supported:
- * <ul>
- * <li>url - a URL template</li>
- * <li>payload - string, optional payload template. If absent a JSON map of all properties listed above will be used.</li>
- * <li>contentType - string, optional payload content type. If absent then <code>application/json</code> will be used.</li>
- * <li>header.* - string, optional header template(s). The name of the property without "header." prefix defines the literal header name.</li>
- * <li>timeout - int, optional connection and socket timeout in milliseconds. Default is 60 seconds.</li>
- * <li>followRedirects - boolean, optional setting to follow redirects. Default is false.</li>
- * </ul>
- */
-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<String, String> 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<String, String> 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 6482542..0000000
--- 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<String, Object> 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<String, Object> results = new LinkedHashMap<>();
- Set<String> 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<String> cleanedUp) throws Exception {
- DistribStateManager stateManager = cloudManager.getDistribStateManager();
- if (!stateManager.hasData(path)) {
- return;
- }
- List<String> markers = stateManager.listData(path);
- markers.forEach(m -> {
- String markerPath = path + "/" + m;
- try {
- Map<String, Object> 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 d3de649..0000000
--- 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.
- * <p>Shard delete requests are put into the {@link ActionContext}'s properties
- * with the key name "operations". The value is a List of SolrRequest objects.</p>
- */
-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<String, Object> 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<String, List<String>> cleanup = new LinkedHashMap<>();
- Map<String, List<String>> inactive = new LinkedHashMap<>();
- Map<String, Map<String, Object>> 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<SolrRequest> operations = (List<SolrRequest>)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<String> 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<String, Object> 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<String, Object> 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 e987828..0000000
--- 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<String> collections = new HashSet<>();
- private final Map<String, Long> lastAboveEventMap = new ConcurrentHashMap<>();
- private final Map<String, Long> 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<String, Object> 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<String, Object> getState() {
- Map<String, Object> state = new HashMap<>();
- state.put("lastAboveEventMap", lastAboveEventMap);
- state.put("lastBelowEventMap", lastBelowEventMap);
- return state;
- }
-
- @Override
- @SuppressWarnings({"unchecked"})
- protected void setState(Map<String, Object> state) {
- this.lastAboveEventMap.clear();
- this.lastBelowEventMap.clear();
- Map<String, Long> replicaVsTime = (Map<String, Long>)state.get("lastAboveEventMap");
- if (replicaVsTime != null) {
- this.lastAboveEventMap.putAll(replicaVsTime);
- }
- replicaVsTime = (Map<String, Long>)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<String, Replica> currentSizes = new HashMap<>();
-
- try {
- ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
- for (String node : clusterState.getLiveNodes()) {
- Map<String, Replica> metricTags = new HashMap<>();
- // coll, shard, replica
- Map<String, Map<String, List<Replica>>> 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<String, Object> 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<String, List<Replica>> aboveSize = new HashMap<>();
-
- Set<String> 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<Replica> 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<String, List<Replica>> 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<Replica> 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<TriggerEvent.Op> 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<String, Object> 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<String, Long> 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<Op> ops, Map<String, List<Replica>> aboveSize,
- Map<String, List<Replica>> 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<String, String> 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<String, String> 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 a7dcf63..0000000
--- 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 573ac77..0000000
--- 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<String, Long> 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<String, Object> 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<String, Object> getState() {
- return null;
- }
-
- @Override
- protected void setState(Map<String, Object> state) {
- lastNodeEvent.clear();
- @SuppressWarnings({"unchecked"})
- Map<String, Long> nodeTimes = (Map<String, Long>) 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<String> liveNodes = null;
- if (node.equals(Policy.ANY)) {
- if (collection.equals(Policy.ANY)) {
- liveNodes = cloudManager.getClusterStateProvider().getLiveNodes();
- } else {
- final Set<String> 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<String, Number> rates = new HashMap<>(liveNodes.size());
- for (String node : liveNodes) {
- Map<String, Object> 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<String, Number> 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<String, Long> 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<String, Number> 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<Op> 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 42188e4..0000000
--- 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<String> lastLiveNodes = new HashSet<>();
-
- private Map<String, Long> 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<String> added = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
- added.forEach(n -> {
- String markerPath = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + n;
- try {
- Map<String, Object> 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<String, Object> 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<String, Object> getState() {
- Map<String,Object> state = new HashMap<>();
- state.put("lastLiveNodes", lastLiveNodes);
- state.put("nodeNameVsTimeAdded", nodeNameVsTimeAdded);
- return state;
- }
-
- @Override
- protected void setState(Map<String, Object> state) {
- this.lastLiveNodes.clear();
- this.nodeNameVsTimeAdded.clear();
- @SuppressWarnings({"unchecked"})
- Collection<String> lastLiveNodes = (Collection<String>)state.get("lastLiveNodes");
- if (lastLiveNodes != null) {
- this.lastLiveNodes.addAll(lastLiveNodes);
- }
- @SuppressWarnings({"unchecked"})
- Map<String,Long> nodeNameVsTimeAdded = (Map<String,Long>)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<String> 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<String> trackingKeySet = nodeNameVsTimeAdded.keySet();
- trackingKeySet.retainAll(newLiveNodes);
-
- // have any new nodes been added?
- Set<String> 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<String> nodeNames = new ArrayList<>();
- List<Long> times = new ArrayList<>();
- for (Iterator<Map.Entry<String, Long>> it = nodeNameVsTimeAdded.entrySet().iterator(); it.hasNext(); ) {
- Map.Entry<String, Long> 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<Long> times, List<String> 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 b1c5818..0000000
--- 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<String> lastLiveNodes = new HashSet<>();
-
- private Map<String, Long> 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<String> lost = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
- lost.forEach(n -> {
- String markerPath = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + n;
- try {
- Map<String, Object> 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<String, Object> 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<String, Object> getState() {
- Map<String,Object> state = new HashMap<>();
- state.put("lastLiveNodes", lastLiveNodes);
- state.put("nodeNameVsTimeRemoved", nodeNameVsTimeRemoved);
- return state;
- }
-
- @Override
- protected void setState(Map<String, Object> state) {
- this.lastLiveNodes.clear();
- this.nodeNameVsTimeRemoved.clear();
- @SuppressWarnings({"unchecked"})
- Collection<String> lastLiveNodes = (Collection<String>)state.get("lastLiveNodes");
- if (lastLiveNodes != null) {
- this.lastLiveNodes.addAll(lastLiveNodes);
- }
- @SuppressWarnings({"unchecked"})
- Map<String,Long> nodeNameVsTimeRemoved = (Map<String,Long>)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<String> 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<String> trackingKeySet = nodeNameVsTimeRemoved.keySet();
- trackingKeySet.removeAll(newLiveNodes);
-
- // have any nodes been removed?
- Set<String> 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<String> nodeNames = new ArrayList<>();
- List<Long> times = new ArrayList<>();
- for (Iterator<Map.Entry<String, Long>> it = nodeNameVsTimeRemoved.entrySet().iterator(); it.hasNext(); ) {
- Map.Entry<String, Long> 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<Long> times, List<String> 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 fa27942..0000000
--- 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<String, AutoScaling.Trigger> 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<String, AutoScaling.Trigger> 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<String> 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<String, AutoScaling.Trigger> 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<String> markers = stateManager.listData(path);
- for (String marker : markers) {
- String markerPath = path + "/" + marker;
- try {
- Map<String, Object> 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<String, AutoScaling.Trigger> triggerMap = loadTriggers(triggerFactory, autoScalingConfig);
-
- // remove all active triggers that have been removed from ZK
- Set<String> trackingKeySet = activeTriggers.keySet();
- trackingKeySet.retainAll(triggerMap.keySet());
-
- // now lets add or remove triggers which have been enabled or disabled respectively
- for (Map.Entry<String, AutoScaling.Trigger> entry : triggerMap.entrySet()) {
- String triggerName = entry.getKey();
- AutoScaling.Trigger trigger = entry.getValue();
- if (trigger.isEnabled()) {
- activeTriggers.put(triggerName, trigger);
- } else {
- activeTriggers.remove(triggerName);
- }
- }
- updated.signalAll();
- } finally {
- updateLock.unlock();
- }
- }
-
- private AutoScalingConfig withDefaultPolicy(AutoScalingConfig autoScalingConfig) {
- Policy policy = autoScalingConfig.getPolicy();
- if (policy.hasEmptyClusterPolicy()) {
- policy = policy.withClusterPolicy(Policy.DEFAULT_CLUSTER_POLICY);
- autoScalingConfig = autoScalingConfig.withPolicy(policy);
- }
- return autoScalingConfig;
- }
-
- private AutoScalingConfig withAutoAddReplicasTrigger(AutoScalingConfig autoScalingConfig) {
- Map<String, Object> triggerProps = AutoScaling.AUTO_ADD_REPLICAS_TRIGGER_PROPS;
- return withDefaultTrigger(triggerProps, autoScalingConfig);
- }
-
- private AutoScalingConfig withScheduledMaintenanceTrigger(AutoScalingConfig autoScalingConfig) {
- Map<String, Object> triggerProps = AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_PROPS;
- return withDefaultTrigger(triggerProps, autoScalingConfig);
- }
-
- private AutoScalingConfig withDefaultTrigger(Map<String, Object> triggerProps, AutoScalingConfig autoScalingConfig) {
- String triggerName = (String) triggerProps.get("name");
- Map<String, AutoScalingConfig.TriggerConfig> configs = autoScalingConfig.getTriggerConfigs();
- for (AutoScalingConfig.TriggerConfig cfg : configs.values()) {
- if (triggerName.equals(cfg.name)) {
- // already has this trigger
- return autoScalingConfig;
- }
- }
- // need to add
- triggerProps.computeIfPresent("waitFor", (k, v) -> (long) (DEFAULT_AUTO_ADD_REPLICA_WAIT_FOR_SECONDS));
- AutoScalingConfig.TriggerConfig config = new AutoScalingConfig.TriggerConfig(triggerName, triggerProps);
- autoScalingConfig = autoScalingConfig.withTriggerConfig(config);
- // need to add SystemLogListener explicitly here
- autoScalingConfig = AutoScalingHandler.withSystemLogListener(autoScalingConfig, triggerName);
- return autoScalingConfig;
- }
-
- private static Map<String, AutoScaling.Trigger> loadTriggers(AutoScaling.TriggerFactory triggerFactory, AutoScalingConfig autoScalingConfig) {
- Map<String, AutoScalingConfig.TriggerConfig> triggers = autoScalingConfig.getTriggerConfigs();
- if (triggers == null) {
- return Collections.emptyMap();
- }
-
- Map<String, AutoScaling.Trigger> triggerMap = new HashMap<>(triggers.size());
-
- for (Map.Entry<String, AutoScalingConfig.TriggerConfig> entry : triggers.entrySet()) {
- AutoScalingConfig.TriggerConfig cfg = entry.getValue();
- TriggerEventType eventType = cfg.event;
- String triggerName = entry.getKey();
- try {
- triggerMap.put(triggerName, triggerFactory.create(eventType, triggerName, cfg.properties));
- } catch (TriggerValidationException e) {
- log.warn("Error in trigger '{}' configuration, trigger config ignored: {}", triggerName, cfg, e);
- }
- }
- return triggerMap;
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
deleted file mode 100644
index 98a367c..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
+++ /dev/null
@@ -1,227 +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.text.ParseException;
-import java.time.Instant;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeFormatterBuilder;
-import java.time.temporal.ChronoField;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-import java.util.TimeZone;
-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.params.AutoScalingParams;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.DateMathParser;
-import org.apache.solr.util.TimeZoneUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
-
-/**
- * A trigger which creates {@link TriggerEventType#SCHEDULED} events as per the configured schedule
- */
-public class ScheduledTrigger extends TriggerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- private static final String DEFAULT_GRACE_DURATION = "+15MINUTES";
- private static final String LAST_RUN_AT = "lastRunAt";
- static final String ACTUAL_EVENT_TIME = "actualEventTime";
-
- private String everyStr;
-
- private String graceDurationStr;
-
- private String preferredOp;
-
- private TimeZone timeZone;
-
- private Instant lastRunAt;
-
- public ScheduledTrigger(String name) {
- super(TriggerEventType.SCHEDULED, name);
- TriggerUtils.requiredProperties(requiredProperties, validProperties, "startTime", "every");
- TriggerUtils.validProperties(validProperties, "timeZone", "graceDuration", AutoScalingParams.PREFERRED_OP);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- String timeZoneStr = (String) properties.get("timeZone");
- this.timeZone = TimeZoneUtils.parseTimezone(timeZoneStr); // defaults to UTC
-
- String startTimeStr = (String) properties.get("startTime");
- this.everyStr = (String) properties.get("every");
- this.graceDurationStr = (String) properties.getOrDefault("graceDuration", DEFAULT_GRACE_DURATION);
-
- preferredOp = (String) properties.get(PREFERRED_OP);
- if (preferredOp != null &&
- CollectionParams.CollectionAction.get(preferredOp) == null) {
- throw new TriggerValidationException(getName(), PREFERRED_OP, "unrecognized value of: '" + preferredOp + "'");
- }
-
- // attempt parsing to validate date math strings
- // explicitly set NOW because it may be different for simulated time
- Date now = new Date(TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getEpochTimeNs()));
- Instant startTime = parseStartTime(now, startTimeStr, timeZoneStr);
- DateMathParser.parseMath(now, startTime + everyStr, timeZone);
- DateMathParser.parseMath(now, startTime + graceDurationStr, timeZone);
-
- // We set lastRunAt to be the startTime (which could be a date math expression such as 'NOW')
- // Ordinarily, NOW will always be evaluated in this constructor so it may seem that
- // the trigger will always fire the first time.
- // However, the lastRunAt is overwritten with the value from ZK
- // during restoreState() operation (which is performed before run()) so the trigger works correctly
- this.lastRunAt = startTime;
- }
-
- private Instant parseStartTime(Date now, String startTimeStr, String timeZoneStr) throws TriggerValidationException {
- try {
- // try parsing startTime as an ISO-8601 date time string
- return DateMathParser.parseMath(now, startTimeStr).toInstant();
- } catch (SolrException e) {
- if (e.code() != SolrException.ErrorCode.BAD_REQUEST.code) {
- throw new TriggerValidationException("startTime", "error parsing value '" + startTimeStr + "': " + e.toString());
- }
- }
- if (timeZoneStr == null) {
- throw new TriggerValidationException("timeZone",
- "Either 'startTime' should be an ISO-8601 date time string or 'timeZone' must be not be null");
- }
- TimeZone timeZone = TimeZone.getTimeZone(timeZoneStr);
- DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
- .append(DateTimeFormatter.ISO_LOCAL_DATE).appendPattern("['T'[HH[:mm[:ss]]]]")
- .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
- .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
- .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
- .toFormatter(Locale.ROOT).withZone(timeZone.toZoneId());
- try {
- return Instant.from(dateTimeFormatter.parse(startTimeStr));
- } catch (Exception e) {
- throw new TriggerValidationException("startTime", "error parsing startTime '" + startTimeStr + "': " + e.toString());
- }
- }
-
- @Override
- protected Map<String, Object> getState() {
- return Collections.singletonMap(LAST_RUN_AT, lastRunAt.toEpochMilli());
- }
-
- @Override
- protected void setState(Map<String, Object> state) {
- if (state.containsKey(LAST_RUN_AT)) {
- this.lastRunAt = Instant.ofEpochMilli((Long) state.get(LAST_RUN_AT));
- }
- }
-
- @Override
- public void restoreState(AutoScaling.Trigger old) {
- assert old.isClosed();
- if (old instanceof ScheduledTrigger) {
- ScheduledTrigger scheduledTrigger = (ScheduledTrigger) old;
- this.lastRunAt = scheduledTrigger.lastRunAt;
- } else {
- throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
- "Unable to restore state from an unknown type of trigger");
- }
- }
-
- @Override
- public void run() {
- synchronized (this) {
- if (isClosed) {
- log.debug("ScheduledTrigger ran but was already closed");
- return;
- }
- }
-
- TimeSource timeSource = cloudManager.getTimeSource();
- DateMathParser dateMathParser = new DateMathParser(timeZone);
- dateMathParser.setNow(new Date(lastRunAt.toEpochMilli()));
- Instant nextRunTime, nextPlusGrace;
- try {
- Date next = dateMathParser.parseMath(everyStr);
- dateMathParser.setNow(next);
- nextPlusGrace = dateMathParser.parseMath(graceDurationStr).toInstant();
- nextRunTime = next.toInstant();
- } catch (ParseException e) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
- "Unable to calculate next run time. lastRan: " + lastRunAt.toString() + " and date math string: " + everyStr, e);
- }
-
- Instant now = Instant.ofEpochMilli(
- TimeUnit.NANOSECONDS.toMillis(timeSource.getEpochTimeNs()));
- AutoScaling.TriggerEventProcessor processor = processorRef.get();
-
- if (now.isBefore(nextRunTime)) {
- return; // it's not time yet
- }
- if (now.isAfter(nextPlusGrace)) {
- // we are past time and we could not run per schedule so skip this event
- if (log.isWarnEnabled()) {
- log.warn("ScheduledTrigger was not able to run event at scheduled time: {}. Now: {}",
- nextRunTime, now);
- }
- // Even though we are skipping the event, we need to notify any listeners of the IGNORED stage
- // so we create a dummy event with the ignored=true flag and ScheduledTriggers will do the rest
- if (processor != null && processor.process(new ScheduledEvent(getEventType(), getName(), timeSource.getTimeNs(),
- preferredOp, now.toEpochMilli(), true))) {
- lastRunAt = nextRunTime;
- return;
- }
- }
-
- if (processor != null) {
- if (log.isDebugEnabled()) {
- log.debug("ScheduledTrigger {} firing registered processor for scheduled time {}, now={}", name,
- nextRunTime, now);
- }
- if (processor.process(new ScheduledEvent(getEventType(), getName(), timeSource.getTimeNs(),
- preferredOp, now.toEpochMilli()))) {
- lastRunAt = nextRunTime; // set to nextRunTime instead of now to avoid drift
- }
- } else {
- lastRunAt = nextRunTime; // set to nextRunTime instead of now to avoid drift
- }
- }
-
- public static class ScheduledEvent extends TriggerEvent {
- public ScheduledEvent(TriggerEventType eventType, String source, long eventTime, String preferredOp, long actualEventTime) {
- this(eventType, source, eventTime, preferredOp, actualEventTime, false);
- }
-
- public ScheduledEvent(TriggerEventType eventType, String source, long eventTime, String preferredOp, long actualEventTime, boolean ignored) {
- super(eventType, source, eventTime, null, ignored);
- if (preferredOp != null) {
- properties.put(PREFERRED_OP, preferredOp);
- }
- properties.put(ACTUAL_EVENT_TIME, actualEventTime);
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
deleted file mode 100644
index e080eec..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
+++ /dev/null
@@ -1,913 +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.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.stream.Collectors;
-
-import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.apache.solr.client.solrj.cloud.DistribStateManager;
-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.TriggerEventProcessorStage;
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest.RequestStatusResponse;
-import org.apache.solr.client.solrj.response.RequestStatusState;
-import org.apache.solr.cloud.Stats;
-import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.cloud.autoscaling.ExecutePlanAction.waitForTaskToFinish;
-import static org.apache.solr.common.params.AutoScalingParams.ACTION_THROTTLE_PERIOD_SECONDS;
-import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_COOLDOWN_PERIOD_SECONDS;
-import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_CORE_POOL_SIZE;
-import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_SCHEDULE_DELAY_SECONDS;
-import static org.apache.solr.common.util.ExecutorUtil.awaitTermination;
-
-/**
- * Responsible for scheduling active triggers, starting and stopping them and
- * performing actions when they fire
- */
-public class ScheduledTriggers implements Closeable {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- public static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
- public static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
- public static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
- public static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
-
- static final Map<String, Object> DEFAULT_PROPERTIES = new HashMap<>();
-
- static {
- DEFAULT_PROPERTIES.put(TRIGGER_SCHEDULE_DELAY_SECONDS, DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
- DEFAULT_PROPERTIES.put(TRIGGER_COOLDOWN_PERIOD_SECONDS, DEFAULT_COOLDOWN_PERIOD_SECONDS);
- DEFAULT_PROPERTIES.put(TRIGGER_CORE_POOL_SIZE, DEFAULT_TRIGGER_CORE_POOL_SIZE);
- DEFAULT_PROPERTIES.put(ACTION_THROTTLE_PERIOD_SECONDS, DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS);
- }
-
- protected static final Random RANDOM;
- static {
- // We try to make things reproducible in the context of our tests by initializing the random instance
- // based on the current seed
- String seed = System.getProperty("tests.seed");
- if (seed == null) {
- RANDOM = new Random();
- } else {
- RANDOM = new Random(seed.hashCode());
- }
- }
-
- private final Map<String, TriggerWrapper> scheduledTriggerWrappers = new ConcurrentHashMap<>();
-
- /**
- * Thread pool for scheduling the triggers
- */
- private final ScheduledThreadPoolExecutor scheduledThreadPoolExecutor;
-
- /**
- * Single threaded executor to run the actions upon a trigger event. We rely on this being a single
- * threaded executor to ensure that trigger fires do not step on each other as well as to ensure
- * that we do not run scheduled trigger threads while an action has been submitted to this executor
- */
- private final ExecutorService actionExecutor;
-
- private boolean isClosed = false;
-
- private final AtomicBoolean hasPendingActions = new AtomicBoolean(false);
-
- private final AtomicLong cooldownStart = new AtomicLong();
-
- private final AtomicLong cooldownPeriod = new AtomicLong(TimeUnit.SECONDS.toNanos(DEFAULT_COOLDOWN_PERIOD_SECONDS));
-
- private final AtomicLong triggerDelay = new AtomicLong(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
-
- private final SolrCloudManager cloudManager;
-
- private final DistribStateManager stateManager;
-
- private final SolrResourceLoader loader;
-
- private final Stats queueStats;
-
- private final TriggerListeners listeners;
-
- private final List<TriggerListener> additionalListeners = new ArrayList<>();
-
- private AutoScalingConfig autoScalingConfig;
-
- public ScheduledTriggers(SolrResourceLoader loader, SolrCloudManager cloudManager) {
- scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(DEFAULT_TRIGGER_CORE_POOL_SIZE,
- new SolrNamedThreadFactory("ScheduledTrigger"));
- scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
- scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
- actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("AutoscalingActionExecutor"));
- this.cloudManager = cloudManager;
- this.stateManager = cloudManager.getDistribStateManager();
- this.loader = loader;
- queueStats = new Stats();
- listeners = new TriggerListeners();
- // initialize cooldown timer
- cooldownStart.set(cloudManager.getTimeSource().getTimeNs() - cooldownPeriod.get());
- }
-
- /**
- * Set the current autoscaling config. This is invoked by {@link OverseerTriggerThread} when autoscaling.json is updated,
- * and it re-initializes trigger listeners and other properties used by the framework
- * @param autoScalingConfig current autoscaling.json
- */
- public void setAutoScalingConfig(AutoScalingConfig autoScalingConfig) {
- Map<String, Object> currentProps = new HashMap<>(DEFAULT_PROPERTIES);
- if (this.autoScalingConfig != null) {
- currentProps.putAll(this.autoScalingConfig.getProperties());
- }
-
- // reset listeners early in order to capture first execution of newly scheduled triggers
- listeners.setAutoScalingConfig(autoScalingConfig);
-
- for (Map.Entry<String, Object> entry : currentProps.entrySet()) {
- Map<String, Object> newProps = autoScalingConfig.getProperties();
- String key = entry.getKey();
- if (newProps.containsKey(key) && !entry.getValue().equals(newProps.get(key))) {
- if (log.isDebugEnabled()) {
- log.debug("Changing value of autoscaling property: {} from: {} to: {}", key, entry.getValue(), newProps.get(key));
- }
- switch (key) {
- case TRIGGER_SCHEDULE_DELAY_SECONDS:
- triggerDelay.set(((Number) newProps.get(key)).intValue());
- synchronized (this) {
- scheduledTriggerWrappers.forEach((s, triggerWrapper) -> {
- if (triggerWrapper.scheduledFuture.cancel(false)) {
- triggerWrapper.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(
- triggerWrapper, 0,
- cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
- TimeUnit.MILLISECONDS);
- } else {
- log.debug("Failed to cancel scheduled task: {}", s);
- }
- });
- }
- break;
- case TRIGGER_COOLDOWN_PERIOD_SECONDS:
- cooldownPeriod.set(TimeUnit.SECONDS.toNanos(((Number) newProps.get(key)).longValue()));
- break;
- case TRIGGER_CORE_POOL_SIZE:
- this.scheduledThreadPoolExecutor.setCorePoolSize(((Number) newProps.get(key)).intValue());
- break;
- }
- }
- }
-
- this.autoScalingConfig = autoScalingConfig;
- // reset cooldown
- cooldownStart.set(cloudManager.getTimeSource().getTimeNs() - cooldownPeriod.get());
- }
-
- /**
- * Adds a new trigger or replaces an existing one. The replaced trigger, if any, is closed
- * <b>before</b> the new trigger is run. If a trigger is replaced with itself then this
- * operation becomes a no-op.
- *
- * @param newTrigger the trigger to be managed
- * @throws AlreadyClosedException if this class has already been closed
- */
- public synchronized void add(AutoScaling.Trigger newTrigger) throws Exception {
- if (isClosed) {
- throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore");
- }
- TriggerWrapper st;
- try {
- st = new TriggerWrapper(newTrigger, cloudManager, queueStats);
- } catch (Exception e) {
- if (isClosed || e instanceof AlreadyClosedException) {
- throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore");
- }
- if (cloudManager.isClosed()) {
- log.error("Failed to add trigger {} - closing or disconnected from data provider", newTrigger.getName(), e);
- } else {
- log.error("Failed to add trigger {}", newTrigger.getName(), e);
- }
- return;
- }
- TriggerWrapper triggerWrapper = st;
-
- TriggerWrapper old = scheduledTriggerWrappers.putIfAbsent(newTrigger.getName(), triggerWrapper);
- if (old != null) {
- if (old.trigger.equals(newTrigger)) {
- // the trigger wasn't actually modified so we do nothing
- return;
- }
- IOUtils.closeQuietly(old);
- newTrigger.restoreState(old.trigger);
- triggerWrapper.setReplay(false);
- scheduledTriggerWrappers.replace(newTrigger.getName(), triggerWrapper);
- }
- newTrigger.setProcessor(event -> {
- TriggerListeners triggerListeners = listeners.copy();
- if (cloudManager.isClosed()) {
- String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because Solr has been shutdown.", event.toString());
- log.warn(msg);
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
- return false;
- }
- TriggerWrapper scheduledSource = scheduledTriggerWrappers.get(event.getSource());
- if (scheduledSource == null) {
- String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the source trigger: %s doesn't exist.", event.toString(), event.getSource());
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.FAILED, msg);
- log.warn(msg);
- return false;
- }
- boolean replaying = event.getProperty(TriggerEvent.REPLAYING) != null ? (Boolean)event.getProperty(TriggerEvent.REPLAYING) : false;
- AutoScaling.Trigger source = scheduledSource.trigger;
- if (scheduledSource.isClosed || source.isClosed()) {
- String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the source trigger: %s has already been closed", event.toString(), source);
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
- log.warn(msg);
- // we do not want to lose this event just because the trigger was closed, perhaps a replacement will need it
- return false;
- }
- if (event.isIgnored()) {
- log.debug("-------- Ignoring event: {}", event);
- event.getProperties().put(TriggerEvent.IGNORED, true);
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "Event was ignored.");
- return true; // always return true for ignored events
- }
- // even though we pause all triggers during action execution there is a possibility that a trigger was already
- // running at the time and would have already created an event so we reject such events during cooldown period
- if (cooldownStart.get() + cooldownPeriod.get() > cloudManager.getTimeSource().getTimeNs()) {
- log.debug("-------- Cooldown period - rejecting event: {}", event);
- event.getProperties().put(TriggerEvent.COOLDOWN, true);
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "In cooldown period.");
- return false;
- } else {
- log.debug("++++++++ Cooldown inactive - processing event: {}", event);
- // start cooldown here to immediately reject other events
- cooldownStart.set(cloudManager.getTimeSource().getTimeNs());
- }
- if (hasPendingActions.compareAndSet(false, true)) {
- // pause all triggers while we execute actions so triggers do not operate on a cluster in transition
- pauseTriggers();
-
- final boolean enqueued;
- if (replaying) {
- enqueued = false;
- } else {
- enqueued = triggerWrapper.enqueue(event);
- }
- // fire STARTED event listeners after enqueuing the event is successful
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.STARTED);
- List<TriggerAction> actions = source.getActions();
- if (actions != null) {
- if (actionExecutor.isShutdown()) {
- String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s from trigger %s because the executor has already been closed", event.toString(), source);
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
- log.warn(msg);
- hasPendingActions.set(false);
- // we do not want to lose this event just because the trigger was closed, perhaps a replacement will need it
- return false;
- }
- actionExecutor.submit(() -> {
- assert hasPendingActions.get();
- long eventProcessingStart = cloudManager.getTimeSource().getTimeNs();
- TriggerListeners triggerListeners1 = triggerListeners.copy();
- log.debug("-- processing actions for {}", event);
- try {
- // in future, we could wait for pending tasks in a different thread and re-enqueue
- // this event so that we continue processing other events and not block this action executor
- waitForPendingTasks(newTrigger, actions);
-
- ActionContext actionContext = new ActionContext(cloudManager, newTrigger, new HashMap<>());
- for (TriggerAction action : actions) {
- @SuppressWarnings({"unchecked"})
- List<String> beforeActions = (List<String>) actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.BEFORE_ACTION.toString(), k -> new ArrayList<String>());
- beforeActions.add(action.getName());
- triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.BEFORE_ACTION, action.getName(), actionContext);
- try {
- action.process(event, actionContext);
- } catch (Exception e) {
- triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.FAILED, action.getName(), actionContext, e, null);
- throw new TriggerActionException(event.getSource(), action.getName(), "Error processing action for trigger event: " + event, e);
- }
- @SuppressWarnings({"unchecked"})
- List<String> afterActions = (List<String>) actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.AFTER_ACTION.toString(), k -> new ArrayList<String>());
- afterActions.add(action.getName());
- triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.AFTER_ACTION, action.getName(), actionContext);
- }
- if (enqueued) {
- TriggerEvent ev = triggerWrapper.dequeue();
- assert ev.getId().equals(event.getId());
- }
- triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.SUCCEEDED);
- } catch (TriggerActionException e) {
- log.warn("Exception executing actions", e);
- } catch (Exception e) {
- triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.FAILED);
- log.warn("Unhandled exception executing actions", e);
- } finally {
- // update cooldown to the time when we actually finished processing the actions
- cooldownStart.set(cloudManager.getTimeSource().getTimeNs());
- hasPendingActions.set(false);
- // resume triggers after cool down period
- resumeTriggers(cloudManager.getTimeSource().convertDelay(TimeUnit.NANOSECONDS, cooldownPeriod.get(), TimeUnit.MILLISECONDS));
- }
- if (log.isDebugEnabled()) {
- log.debug("-- processing took {} ms for event id={}",
- TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getTimeNs() - eventProcessingStart), event.id);
- }
- });
- } else {
- if (enqueued) {
- TriggerEvent ev = triggerWrapper.dequeue();
- if (!ev.getId().equals(event.getId())) {
- throw new RuntimeException("Wrong event dequeued, queue of " + triggerWrapper.trigger.getName()
- + " is broken! Expected event=" + event + " but got " + ev);
- }
- }
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.SUCCEEDED);
- hasPendingActions.set(false);
- // resume triggers now
- resumeTriggers(0);
- }
- return true;
- } else {
- log.debug("Ignoring event {}, already processing other actions.", event.id);
- // there is an action in the queue and we don't want to enqueue another until it is complete
- triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "Already processing another event.");
- return false;
- }
- });
- newTrigger.init(); // mark as ready for scheduling
- triggerWrapper.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(triggerWrapper, 0,
- cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
- TimeUnit.MILLISECONDS);
- }
-
- /**
- * Pauses all scheduled trigger invocations without interrupting any that are in progress
- * @lucene.internal
- */
- public synchronized void pauseTriggers() {
- if (log.isDebugEnabled()) {
- log.debug("Pausing all triggers: {}", scheduledTriggerWrappers.keySet());
- }
- scheduledTriggerWrappers.forEach((s, triggerWrapper) -> triggerWrapper.scheduledFuture.cancel(false));
- }
-
- /**
- * Resumes all previously cancelled triggers to be scheduled after the given initial delay
- * @param afterDelayMillis the initial delay in milliseconds after which triggers should be resumed
- * @lucene.internal
- */
- public synchronized void resumeTriggers(long afterDelayMillis) {
- List<Map.Entry<String, TriggerWrapper>> entries = new ArrayList<>(scheduledTriggerWrappers.entrySet());
- Collections.shuffle(entries, RANDOM);
- entries.forEach(e -> {
- String key = e.getKey();
- TriggerWrapper triggerWrapper = e.getValue();
- if (triggerWrapper.scheduledFuture.isCancelled()) {
- log.debug("Resuming trigger: {} after {}ms", key, afterDelayMillis);
- triggerWrapper.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(triggerWrapper, afterDelayMillis,
- cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
- }
- });
- }
-
- private void waitForPendingTasks(AutoScaling.Trigger newTrigger, List<TriggerAction> actions) throws AlreadyClosedException {
- DistribStateManager stateManager = cloudManager.getDistribStateManager();
- try {
-
- for (TriggerAction action : actions) {
- if (action instanceof ExecutePlanAction) {
- String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + newTrigger.getName() + "/" + action.getName();
- if (!stateManager.hasData(parentPath)) {
- break;
- }
- List<String> children = stateManager.listData(parentPath);
- if (children != null) {
- for (String child : children) {
- String path = parentPath + '/' + child;
- VersionedData data = stateManager.getData(path, null);
- if (data != null) {
- @SuppressWarnings({"rawtypes"})
- Map map = (Map) Utils.fromJSON(data.getData());
- String requestid = (String) map.get("requestid");
- try {
- log.debug("Found pending task with requestid={}", requestid);
- RequestStatusResponse statusResponse = waitForTaskToFinish(cloudManager, requestid,
- ExecutePlanAction.DEFAULT_TASK_TIMEOUT_SECONDS, TimeUnit.SECONDS);
- if (statusResponse != null) {
- RequestStatusState state = statusResponse.getRequestStatus();
- if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED || state == RequestStatusState.NOT_FOUND) {
- stateManager.removeData(path, -1);
- }
- }
- } catch (Exception e) {
- if (cloudManager.isClosed()) {
- throw e; // propagate the abort to the caller
- }
- 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;
- }
- log.error("Unexpected exception while waiting for pending task with requestid: {} to finish", requestid, e);
- }
- }
- }
- }
- }
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Thread interrupted", e);
- } catch (Exception e) {
- if (cloudManager.isClosed()) {
- throw new AlreadyClosedException("The Solr instance has been shutdown");
- }
- // we catch but don't rethrow because a failure to wait for pending tasks
- // should not keep the actions from executing
- log.error("Unexpected exception while waiting for pending tasks to finish", e);
- }
- }
-
- /**
- * Remove and stop all triggers. Also cleans up any leftover
- * state / events in ZK.
- */
- public synchronized void removeAll() {
- getScheduledTriggerNames().forEach(t -> {
- log.info("-- removing trigger: {}", t);
- remove(t);
- });
- }
-
- /**
- * Removes and stops the trigger with the given name. Also cleans up any leftover
- * state / events in ZK.
- *
- * @param triggerName the name of the trigger to be removed
- */
- public synchronized void remove(String triggerName) {
- TriggerWrapper removed = scheduledTriggerWrappers.remove(triggerName);
- IOUtils.closeQuietly(removed);
- removeTriggerZKData(triggerName);
- }
-
- private void removeTriggerZKData(String triggerName) {
- String statePath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + triggerName;
- String eventsPath = ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName;
- try {
- stateManager.removeRecursively(statePath, true, true);
- } catch (Exception e) {
- log.warn("Failed to remove state for removed trigger {}", statePath, e);
- }
- try {
- stateManager.removeRecursively(eventsPath, true, true);
- } catch (Exception e) {
- log.warn("Failed to remove events for removed trigger {}", eventsPath, e);
- }
- }
-
- /**
- * @return an unmodifiable set of names of all triggers being managed by this class
- */
- public synchronized Set<String> getScheduledTriggerNames() {
- return Set.copyOf(scheduledTriggerWrappers.keySet()); // shallow copy
- }
-
- /**
- * For use in white/grey box testing: The Trigger returned may be inspected,
- * but should not be modified in any way.
- *
- * @param name the name of an existing trigger
- * @return the current scheduled trigger with that name, or null if none exists
- * @lucene.internal
- */
- public synchronized AutoScaling.Trigger getTrigger(String name) {
- TriggerWrapper w = scheduledTriggerWrappers.get(name);
- return (null == w) ? null : w.trigger;
- }
-
- @Override
- public void close() throws IOException {
- synchronized (this) {
- // mark that we are closed
- isClosed = true;
- for (TriggerWrapper triggerWrapper : scheduledTriggerWrappers.values()) {
- IOUtils.closeQuietly(triggerWrapper);
- }
- scheduledTriggerWrappers.clear();
- }
- // shutdown and interrupt all running tasks because there's no longer any
- // guarantee about cluster state
- log.debug("Shutting down scheduled thread pool executor now");
- scheduledThreadPoolExecutor.shutdownNow();
-
- log.debug("Shutting down action executor now");
- actionExecutor.shutdownNow();
-
- listeners.close();
-
- log.debug("Awaiting termination for action executor");
- awaitTermination(actionExecutor);
-
- log.debug("Awaiting termination for scheduled thread pool executor");
- awaitTermination(scheduledThreadPoolExecutor);
-
- log.debug("ScheduledTriggers closed completely");
- }
-
- /**
- * Add a temporary listener for internal use (tests, simulation).
- * @param listener listener instance
- */
- public void addAdditionalListener(TriggerListener listener) {
- listeners.addAdditionalListener(listener);
- }
-
- /**
- * Remove a temporary listener for internal use (tests, simulation).
- * @param listener listener instance
- */
- public void removeAdditionalListener(TriggerListener listener) {
- listeners.removeAdditionalListener(listener);
- }
-
- private class TriggerWrapper implements Runnable, Closeable {
- AutoScaling.Trigger trigger;
- ScheduledFuture<?> scheduledFuture;
- TriggerEventQueue queue;
- boolean replay;
- volatile boolean isClosed;
-
- TriggerWrapper(AutoScaling.Trigger trigger, SolrCloudManager cloudManager, Stats stats) throws IOException {
- this.trigger = trigger;
- this.queue = new TriggerEventQueue(cloudManager, trigger.getName(), stats);
- this.replay = true;
- this.isClosed = false;
- }
-
- public void setReplay(boolean replay) {
- this.replay = replay;
- }
-
- public boolean enqueue(TriggerEvent event) {
- if (isClosed) {
- throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
- }
- return queue.offerEvent(event);
- }
-
- public TriggerEvent dequeue() {
- if (isClosed) {
- throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
- }
- TriggerEvent event = queue.pollEvent();
- return event;
- }
-
- @Override
- public void run() {
- if (isClosed) {
- throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
- }
- // fire a trigger only if an action is not pending
- // note this is not fool proof e.g. it does not prevent an action being executed while a trigger
- // is still executing. There is additional protection against that scenario in the event listener.
- if (!hasPendingActions.get()) {
- // this synchronization is usually never under contention
- // but the only reason to have it here is to ensure that when the set-properties API is used
- // to change the schedule delay, we can safely cancel the old scheduled task
- // and create another one with the new delay without worrying about concurrent
- // execution of the same trigger instance
- synchronized (TriggerWrapper.this) {
- // replay accumulated events on first run, if any
-
- try {
- if (replay) {
- TriggerEvent event;
- // peek first without removing - we may crash before calling the listener
- while ((event = queue.peekEvent()) != null) {
- // override REPLAYING=true
- event.getProperties().put(TriggerEvent.REPLAYING, true);
- if (!trigger.getProcessor().process(event)) {
- log.error("Failed to re-play event, discarding: {}", event);
- }
- queue.pollEvent(); // always remove it from queue
- }
- // now restore saved state to possibly generate new events from old state on the first run
- try {
- trigger.restoreState();
- } catch (Exception e) {
- // log but don't throw - see below
- log.error("Error restoring trigger state {}", trigger.getName(), e);
- }
- replay = false;
- }
- } catch (AlreadyClosedException e) {
-
- } catch (Exception e) {
- log.error("Unexpected exception from trigger: {}", trigger.getName(), e);
- }
- try {
- trigger.run();
- } catch (AlreadyClosedException e) {
-
- } catch (Exception e) {
- // log but do not propagate exception because an exception thrown from a scheduled operation
- // will suppress future executions
- log.error("Unexpected exception from trigger: {}", trigger.getName(), e);
- } finally {
- // checkpoint after each run
- trigger.saveState();
- }
- }
- }
- }
-
- @Override
- public void close() throws IOException {
- isClosed = true;
- if (scheduledFuture != null) {
- scheduledFuture.cancel(true);
- }
- IOUtils.closeQuietly(trigger);
- }
- }
-
- private class TriggerListeners {
- Map<String, Map<TriggerEventProcessorStage, List<TriggerListener>>> listenersPerStage = new HashMap<>();
- Map<String, TriggerListener> listenersPerName = new HashMap<>();
- List<TriggerListener> additionalListeners = new ArrayList<>();
- ReentrantLock updateLock = new ReentrantLock();
-
- public TriggerListeners() {
-
- }
-
- private TriggerListeners(Map<String, Map<TriggerEventProcessorStage, List<TriggerListener>>> listenersPerStage,
- Map<String, TriggerListener> listenersPerName) {
- this.listenersPerStage = new HashMap<>();
- listenersPerStage.forEach((n, listeners) -> {
- Map<TriggerEventProcessorStage, List<TriggerListener>> perStage = this.listenersPerStage.computeIfAbsent(n, name -> new HashMap<>());
- listeners.forEach((s, lst) -> {
- List<TriggerListener> newLst = perStage.computeIfAbsent(s, stage -> new ArrayList<>());
- newLst.addAll(lst);
- });
- });
- this.listenersPerName = new HashMap<>(listenersPerName);
- }
-
- public TriggerListeners copy() {
- return new TriggerListeners(listenersPerStage, listenersPerName);
- }
-
- public void addAdditionalListener(TriggerListener listener) {
- updateLock.lock();
- try {
- AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
- for (TriggerEventProcessorStage stage : config.stages) {
- addPerStage(config.trigger, stage, listener);
- }
- // add also for beforeAction / afterAction TriggerStage
- if (!config.beforeActions.isEmpty()) {
- addPerStage(config.trigger, TriggerEventProcessorStage.BEFORE_ACTION, listener);
- }
- if (!config.afterActions.isEmpty()) {
- addPerStage(config.trigger, TriggerEventProcessorStage.AFTER_ACTION, listener);
- }
- additionalListeners.add(listener);
- } finally {
- updateLock.unlock();
- }
- }
-
- public void removeAdditionalListener(TriggerListener listener) {
- updateLock.lock();
- try {
- listenersPerName.remove(listener.getConfig().name);
- listenersPerStage.forEach((trigger, perStage) -> {
- perStage.forEach((stage, listeners) -> {
- listeners.remove(listener);
- });
- });
- additionalListeners.remove(listener);
- } finally {
- updateLock.unlock();
- }
- }
-
- void setAutoScalingConfig(AutoScalingConfig autoScalingConfig) {
- updateLock.lock();
- // we will recreate this from scratch
- listenersPerStage.clear();
- try {
- Set<String> triggerNames = autoScalingConfig.getTriggerConfigs().keySet();
- Map<String, AutoScalingConfig.TriggerListenerConfig> configs = autoScalingConfig.getTriggerListenerConfigs();
- Set<String> listenerNames = configs.entrySet().stream().map(entry -> entry.getValue().name).collect(Collectors.toSet());
- // close those for non-existent triggers and nonexistent listener configs
- for (Iterator<Map.Entry<String, TriggerListener>> it = listenersPerName.entrySet().iterator(); it.hasNext(); ) {
- Map.Entry<String, TriggerListener> entry = it.next();
- String name = entry.getKey();
- TriggerListener listener = entry.getValue();
- if (!triggerNames.contains(listener.getConfig().trigger) || !listenerNames.contains(name)) {
- try {
- listener.close();
- } catch (Exception e) {
- log.warn("Exception closing old listener {}", listener.getConfig(), e);
- }
- it.remove();
- }
- }
- for (Map.Entry<String, AutoScalingConfig.TriggerListenerConfig> entry : configs.entrySet()) {
- AutoScalingConfig.TriggerListenerConfig config = entry.getValue();
- if (!triggerNames.contains(config.trigger)) {
- log.debug("-- skipping listener for non-existent trigger: {}", config);
- continue;
- }
- // find previous instance and reuse if possible
- TriggerListener oldListener = listenersPerName.get(config.name);
- TriggerListener listener = null;
- if (oldListener != null) {
- if (!oldListener.getConfig().equals(config)) { // changed config
- try {
- oldListener.close();
- } catch (Exception e) {
- log.warn("Exception closing old listener {}", oldListener.getConfig(), e);
- }
- } else {
- listener = oldListener; // reuse
- }
- }
- if (listener == null) { // create new instance
- String clazz = config.listenerClass;
- try {
- listener = loader.newInstance(clazz, TriggerListener.class);
- } catch (Exception e) {
- log.warn("Invalid TriggerListener class name '{}', skipping...", clazz, e);
- }
- if (listener != null) {
- try {
- listener.configure(loader, cloudManager, config);
- listener.init();
- listenersPerName.put(config.name, listener);
- } catch (Exception e) {
- log.warn("Error initializing TriggerListener {}", config, e);
- IOUtils.closeQuietly(listener);
- listener = null;
- }
- }
- }
- if (listener == null) {
- continue;
- }
- // add per stage
- for (TriggerEventProcessorStage stage : config.stages) {
- addPerStage(config.trigger, stage, listener);
- }
- // add also for beforeAction / afterAction TriggerStage
- if (!config.beforeActions.isEmpty()) {
- addPerStage(config.trigger, TriggerEventProcessorStage.BEFORE_ACTION, listener);
- }
- if (!config.afterActions.isEmpty()) {
- addPerStage(config.trigger, TriggerEventProcessorStage.AFTER_ACTION, listener);
- }
- }
- // re-add additional listeners
- List<TriggerListener> additional = new ArrayList<>(additionalListeners);
- additionalListeners.clear();
- for (TriggerListener listener : additional) {
- addAdditionalListener(listener);
- }
-
- } finally {
- updateLock.unlock();
- }
- }
-
- private void addPerStage(String triggerName, TriggerEventProcessorStage stage, TriggerListener listener) {
- Map<TriggerEventProcessorStage, List<TriggerListener>> perStage =
- listenersPerStage.computeIfAbsent(triggerName, k -> new HashMap<>());
- List<TriggerListener> lst = perStage.computeIfAbsent(stage, k -> new ArrayList<>(3));
- lst.add(listener);
- }
-
- void reset() {
- updateLock.lock();
- try {
- listenersPerStage.clear();
- for (TriggerListener listener : listenersPerName.values()) {
- IOUtils.closeQuietly(listener);
- }
- listenersPerName.clear();
- } finally {
- updateLock.unlock();
- }
- }
-
- void close() {
- reset();
- }
-
- List<TriggerListener> getTriggerListeners(String trigger, TriggerEventProcessorStage stage) {
- Map<TriggerEventProcessorStage, List<TriggerListener>> perStage = listenersPerStage.get(trigger);
- if (perStage == null) {
- return Collections.emptyList();
- }
- List<TriggerListener> lst = perStage.get(stage);
- if (lst == null) {
- return Collections.emptyList();
- } else {
- return Collections.unmodifiableList(lst);
- }
- }
-
- void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage) {
- fireListeners(trigger, event, stage, null, null, null, null);
- }
-
- void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage, String message) {
- fireListeners(trigger, event, stage, null, null, null, message);
- }
-
- void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
- ActionContext context) {
- fireListeners(trigger, event, stage, actionName, context, null, null);
- }
-
- void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
- ActionContext context, Throwable error, String message) {
- updateLock.lock();
- try {
- for (TriggerListener listener : getTriggerListeners(trigger, stage)) {
- if (!listener.isEnabled()) {
- continue;
- }
- if (actionName != null) {
- AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
- if (stage == TriggerEventProcessorStage.BEFORE_ACTION) {
- if (!config.beforeActions.contains(actionName)) {
- continue;
- }
- } else if (stage == TriggerEventProcessorStage.AFTER_ACTION) {
- if (!config.afterActions.contains(actionName)) {
- continue;
- }
- }
- }
- try {
- listener.onEvent(event, stage, actionName, context, error, message);
- } catch (Exception e) {
- log.warn("Exception running listener {}", listener.getConfig(), e);
- }
- }
- } finally {
- updateLock.unlock();
- }
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
deleted file mode 100644
index 505c511..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
+++ /dev/null
@@ -1,805 +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.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.AtomicDouble;
-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.Replica;
-import org.apache.solr.common.cloud.ZkStateReader;
-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.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.metrics.SolrCoreMetricManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Trigger for the {@link org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType#SEARCHRATE} event.
- */
-public class SearchRateTrigger extends TriggerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public static final String COLLECTIONS_PROP = "collections";
- public static final String METRIC_PROP = "metric";
- public static final String MAX_OPS_PROP = "maxOps";
- public static final String MIN_REPLICAS_PROP = "minReplicas";
- public static final String ABOVE_RATE_PROP = "aboveRate";
- public static final String BELOW_RATE_PROP = "belowRate";
- public static final String ABOVE_NODE_RATE_PROP = "aboveNodeRate";
- public static final String BELOW_NODE_RATE_PROP = "belowNodeRate";
- public static final String ABOVE_OP_PROP = "aboveOp";
- public static final String BELOW_OP_PROP = "belowOp";
- public static final String ABOVE_NODE_OP_PROP = "aboveNodeOp";
- public static final String BELOW_NODE_OP_PROP = "belowNodeOp";
-
- // back-compat
- public static final String BC_COLLECTION_PROP = "collection";
- public static final String BC_RATE_PROP = "rate";
-
-
- public static final String HOT_NODES = "hotNodes";
- public static final String HOT_COLLECTIONS = "hotCollections";
- public static final String HOT_SHARDS = "hotShards";
- public static final String HOT_REPLICAS = "hotReplicas";
- public static final String COLD_NODES = "coldNodes";
- public static final String COLD_COLLECTIONS = "coldCollections";
- public static final String COLD_SHARDS = "coldShards";
- public static final String COLD_REPLICAS = "coldReplicas";
- public static final String VIOLATION_PROP = "violationType";
-
- public static final int DEFAULT_MAX_OPS = 3;
- public static final String DEFAULT_METRIC = "QUERY./select.requestTimes:1minRate";
-
- private String metric;
- private int maxOps;
- private Integer minReplicas = null;
- private final Set<String> collections = new HashSet<>();
- private String shard;
- private String node;
- private double aboveRate;
- private double belowRate;
- private double aboveNodeRate;
- private double belowNodeRate;
- private CollectionParams.CollectionAction aboveOp, belowOp, aboveNodeOp, belowNodeOp;
- private final Map<String, Long> lastCollectionEvent = new ConcurrentHashMap<>();
- private final Map<String, Long> lastNodeEvent = new ConcurrentHashMap<>();
- private final Map<String, Long> lastShardEvent = new ConcurrentHashMap<>();
- private final Map<String, Long> lastReplicaEvent = new ConcurrentHashMap<>();
- private final Map<String, Object> state = new HashMap<>();
-
- public SearchRateTrigger(String name) {
- super(TriggerEventType.SEARCHRATE, name);
- this.state.put("lastCollectionEvent", lastCollectionEvent);
- this.state.put("lastNodeEvent", lastNodeEvent);
- this.state.put("lastShardEvent", lastShardEvent);
- this.state.put("lastReplicaEvent", lastReplicaEvent);
- TriggerUtils.validProperties(validProperties,
- COLLECTIONS_PROP, AutoScalingParams.SHARD, AutoScalingParams.NODE,
- METRIC_PROP,
- MAX_OPS_PROP,
- MIN_REPLICAS_PROP,
- ABOVE_OP_PROP,
- BELOW_OP_PROP,
- ABOVE_NODE_OP_PROP,
- BELOW_NODE_OP_PROP,
- ABOVE_RATE_PROP,
- BELOW_RATE_PROP,
- ABOVE_NODE_RATE_PROP,
- BELOW_NODE_RATE_PROP,
- // back-compat props
- BC_COLLECTION_PROP,
- BC_RATE_PROP);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
- super.configure(loader, cloudManager, properties);
- // parse config options
- String collectionsStr = (String)properties.get(COLLECTIONS_PROP);
- if (collectionsStr != null) {
- collections.addAll(StrUtils.splitSmart(collectionsStr, ','));
- }
- // check back-compat collection prop
- collectionsStr = (String)properties.get(BC_COLLECTION_PROP);
- if (collectionsStr != null) {
- if (!collectionsStr.equals(Policy.ANY)) {
- collections.add(collectionsStr);
- }
- }
- shard = (String)properties.getOrDefault(AutoScalingParams.SHARD, Policy.ANY);
- if (!shard.equals(Policy.ANY) && (collections.isEmpty() || collections.size() > 1)) {
- throw new TriggerValidationException(name, AutoScalingParams.SHARD, "When 'shard' is other than #ANY then exactly one collection name must be set");
- }
- node = (String)properties.getOrDefault(AutoScalingParams.NODE, Policy.ANY);
- metric = (String)properties.getOrDefault(METRIC_PROP, DEFAULT_METRIC);
-
- String maxOpsStr = String.valueOf(properties.getOrDefault(MAX_OPS_PROP, DEFAULT_MAX_OPS));
- try {
- maxOps = Integer.parseInt(maxOpsStr);
- } catch (Exception e) {
- throw new TriggerValidationException(name, MAX_OPS_PROP, "invalid value '" + maxOpsStr + "': " + e.toString());
- }
-
- Object o = properties.get(MIN_REPLICAS_PROP);
- if (o != null) {
- try {
- minReplicas = Integer.parseInt(o.toString());
- if (minReplicas < 1) {
- throw new Exception("must be at least 1, or not set to use 'replicationFactor'");
- }
- } catch (Exception e) {
- throw new TriggerValidationException(name, MIN_REPLICAS_PROP, "invalid value '" + o + "': " + e.toString());
- }
- }
-
- Object above = properties.get(ABOVE_RATE_PROP);
- Object below = properties.get(BELOW_RATE_PROP);
- // back-compat rate prop
- if (properties.containsKey(BC_RATE_PROP)) {
- above = properties.get(BC_RATE_PROP);
- }
- if (above == null && below == null) {
- throw new TriggerValidationException(name, ABOVE_RATE_PROP, "at least one of '" +
- ABOVE_RATE_PROP + "' or '" + BELOW_RATE_PROP + "' must be set");
- }
- if (above != null) {
- try {
- aboveRate = Double.parseDouble(String.valueOf(above));
- } catch (Exception e) {
- throw new TriggerValidationException(name, ABOVE_RATE_PROP, "Invalid configuration value: '" + above + "': " + e.toString());
- }
- } else {
- aboveRate = Double.MAX_VALUE;
- }
- if (below != null) {
- try {
- belowRate = Double.parseDouble(String.valueOf(below));
- } catch (Exception e) {
- throw new TriggerValidationException(name, BELOW_RATE_PROP, "Invalid configuration value: '" + below + "': " + e.toString());
- }
- } else {
- belowRate = -1;
- }
-
- // node rates
- above = properties.get(ABOVE_NODE_RATE_PROP);
- below = properties.get(BELOW_NODE_RATE_PROP);
- if (above != null) {
- try {
- aboveNodeRate = Double.parseDouble(String.valueOf(above));
- } catch (Exception e) {
- throw new TriggerValidationException(name, ABOVE_NODE_RATE_PROP, "Invalid configuration value: '" + above + "': " + e.toString());
- }
- } else {
- aboveNodeRate = Double.MAX_VALUE;
- }
- if (below != null) {
- try {
- belowNodeRate = Double.parseDouble(String.valueOf(below));
- } catch (Exception e) {
- throw new TriggerValidationException(name, BELOW_NODE_RATE_PROP, "Invalid configuration value: '" + below + "': " + e.toString());
- }
- } else {
- belowNodeRate = -1;
- }
-
- String aboveOpStr = String.valueOf(properties.getOrDefault(ABOVE_OP_PROP, CollectionParams.CollectionAction.ADDREPLICA.toLower()));
- String belowOpStr = String.valueOf(properties.getOrDefault(BELOW_OP_PROP, CollectionParams.CollectionAction.DELETEREPLICA.toLower()));
- aboveOp = CollectionParams.CollectionAction.get(aboveOpStr);
- if (aboveOp == null) {
- throw new TriggerValidationException(getName(), ABOVE_OP_PROP, "unrecognized value: '" + aboveOpStr + "'");
- }
- belowOp = CollectionParams.CollectionAction.get(belowOpStr);
- if (belowOp == null) {
- throw new TriggerValidationException(getName(), BELOW_OP_PROP, "unrecognized value: '" + belowOpStr + "'");
- }
- Object aboveNodeObj = properties.getOrDefault(ABOVE_NODE_OP_PROP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
- // do NOT set the default to DELETENODE
- Object belowNodeObj = properties.get(BELOW_NODE_OP_PROP);
- try {
- aboveNodeOp = CollectionParams.CollectionAction.get(String.valueOf(aboveNodeObj));
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), ABOVE_NODE_OP_PROP, "unrecognized value: '" + aboveNodeObj + "'");
- }
- if (belowNodeObj != null) {
- try {
- belowNodeOp = CollectionParams.CollectionAction.get(String.valueOf(belowNodeObj));
- } catch (Exception e) {
- throw new TriggerValidationException(getName(), BELOW_NODE_OP_PROP, "unrecognized value: '" + belowNodeObj + "'");
- }
- }
- }
-
- @VisibleForTesting
- Map<String, Object> getConfig() {
- Map<String, Object> config = new HashMap<>();
- config.put("name", name);
- config.put(COLLECTIONS_PROP, collections);
- config.put(AutoScalingParams.SHARD, shard);
- config.put(AutoScalingParams.NODE, node);
- config.put(METRIC_PROP, metric);
- config.put(MAX_OPS_PROP, maxOps);
- config.put(MIN_REPLICAS_PROP, minReplicas);
- config.put(ABOVE_RATE_PROP, aboveRate);
- config.put(BELOW_RATE_PROP, belowRate);
- config.put(ABOVE_NODE_RATE_PROP, aboveNodeRate);
- config.put(BELOW_NODE_RATE_PROP, belowNodeRate);
- config.put(ABOVE_OP_PROP, aboveOp);
- config.put(ABOVE_NODE_OP_PROP, aboveNodeOp);
- config.put(BELOW_OP_PROP, belowOp);
- config.put(BELOW_NODE_OP_PROP, belowNodeOp);
- return config;
- }
-
- @Override
- protected Map<String, Object> getState() {
- return state;
- }
-
- @Override
- protected void setState(Map<String, Object> state) {
- lastCollectionEvent.clear();
- lastNodeEvent.clear();
- lastShardEvent.clear();
- lastReplicaEvent.clear();
- @SuppressWarnings({"unchecked"})
- Map<String, Long> collTimes = (Map<String, Long>)state.get("lastCollectionEvent");
- if (collTimes != null) {
- lastCollectionEvent.putAll(collTimes);
- }
- @SuppressWarnings({"unchecked"})
- Map<String, Long> nodeTimes = (Map<String, Long>)state.get("lastNodeEvent");
- if (nodeTimes != null) {
- lastNodeEvent.putAll(nodeTimes);
- }
- @SuppressWarnings({"unchecked"})
- Map<String, Long> shardTimes = (Map<String, Long>)state.get("lastShardEvent");
- if (shardTimes != null) {
- lastShardEvent.putAll(shardTimes);
- }
- @SuppressWarnings({"unchecked"})
- Map<String, Long> replicaTimes = (Map<String, Long>)state.get("lastReplicaEvent");
- if (replicaTimes != null) {
- lastReplicaEvent.putAll(replicaTimes);
- }
- }
-
- @Override
- public void restoreState(AutoScaling.Trigger old) {
- assert old.isClosed();
- if (old instanceof SearchRateTrigger) {
- SearchRateTrigger that = (SearchRateTrigger)old;
- assert this.name.equals(that.name);
- this.lastCollectionEvent.clear();
- this.lastNodeEvent.clear();
- this.lastShardEvent.clear();
- this.lastReplicaEvent.clear();
- this.lastCollectionEvent.putAll(that.lastCollectionEvent);
- this.lastNodeEvent.putAll(that.lastNodeEvent);
- this.lastShardEvent.putAll(that.lastShardEvent);
- this.lastReplicaEvent.putAll(that.lastReplicaEvent);
- } 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;
- }
-
- // collection, shard, list(replica + rate)
- Map<String, Map<String, List<Replica>>> collectionRates = new HashMap<>();
- // node, rate
- Map<String, AtomicDouble> nodeRates = new HashMap<>();
- // this replication factor only considers replica types that are searchable
- // collection, shard, RF
- Map<String, Map<String, AtomicInteger>> searchableReplicationFactors = new HashMap<>();
-
- ClusterState clusterState = null;
- try {
- clusterState = cloudManager.getClusterStateProvider().getClusterState();
- } catch (IOException e) {
- log.warn("Error getting ClusterState", e);
- return;
- }
- for (String node : cloudManager.getClusterStateProvider().getLiveNodes()) {
- Map<String, Replica> metricTags = new HashMap<>();
- // coll, shard, replica
- Map<String, Map<String, List<Replica>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
- infos.forEach((coll, shards) -> {
- Map<String, AtomicInteger> replPerShard = searchableReplicationFactors.computeIfAbsent(coll, c -> new HashMap<>());
- shards.forEach((sh, replicas) -> {
- AtomicInteger repl = replPerShard.computeIfAbsent(sh, s -> new AtomicInteger());
- replicas.forEach(replica -> {
- // skip non-active replicas
- if (replica.getState() != Replica.State.ACTIVE) {
- return;
- }
- repl.incrementAndGet();
- // we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
- String replicaName = Utils.parseMetricsReplicaName(coll, replica.getCoreName());
- if (replicaName == null) { // should never happen???
- replicaName = replica.getName(); // which is actually coreNode name...
- }
- String registry = SolrCoreMetricManager.createRegistryName(true, coll, sh, replicaName, null);
- String tag = "metrics:" + registry + ":" + metric;
- metricTags.put(tag, replica);
- });
- });
- });
- if (metricTags.isEmpty()) {
- continue;
- }
- Map<String, Object> rates = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
- if (log.isDebugEnabled()) {
- log.debug("### rates for node {}", node);
- rates.forEach((tag, rate) -> log.debug("### " + tag + "\t" + rate)); // logOk
- }
- rates.forEach((tag, rate) -> {
- Replica info = metricTags.get(tag);
- if (info == null) {
- log.warn("Missing replica info for response tag {}", tag);
- } else {
- Map<String, List<Replica>> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>());
- List<Replica> perShard = perCollection.computeIfAbsent(info.getShard(), s -> new ArrayList<>());
- info = (Replica)info.clone();
- info.getProperties().put(AutoScalingParams.RATE, ((Number)rate).doubleValue());
- perShard.add(info);
- AtomicDouble perNode = nodeRates.computeIfAbsent(node, s -> new AtomicDouble());
- perNode.addAndGet(((Number)rate).doubleValue());
- }
- });
- }
-
- if (log.isDebugEnabled()) {
- collectionRates.forEach((coll, collRates) -> {
- log.debug("## Collection: {}", coll);
- collRates.forEach((s, replicas) -> {
- log.debug("## - {}", s);
- replicas.forEach(ri -> log.debug("## {} {}", ri.getCoreName(), ri.get(AutoScalingParams.RATE))); //logOk
- });
- });
- }
- long now = cloudManager.getTimeSource().getTimeNs();
- Map<String, Double> hotNodes = new HashMap<>();
- Map<String, Double> coldNodes = new HashMap<>();
-
- // check for exceeded rates and filter out those with less than waitFor from previous events
- nodeRates.entrySet().stream()
- .filter(entry -> node.equals(Policy.ANY) || node.equals(entry.getKey()))
- .forEach(entry -> {
- if (entry.getValue().get() > aboveNodeRate) {
- if (waitForElapsed(entry.getKey(), now, lastNodeEvent)) {
- hotNodes.put(entry.getKey(), entry.getValue().get());
- }
- } else if (entry.getValue().get() < belowNodeRate) {
- if (waitForElapsed(entry.getKey(), now, lastNodeEvent)) {
- coldNodes.put(entry.getKey(), entry.getValue().get());
- }
- } else {
- // no violation - clear waitForElapsed
- // (violation is only valid if it persists throughout waitFor)
- lastNodeEvent.remove(entry.getKey());
- }
- });
-
- Map<String, Map<String, Double>> hotShards = new HashMap<>();
- Map<String, Map<String, Double>> coldShards = new HashMap<>();
- List<Replica> hotReplicas = new ArrayList<>();
- List<Replica> coldReplicas = new ArrayList<>();
- collectionRates.forEach((coll, shardRates) -> {
- shardRates.forEach((sh, replicaRates) -> {
- double totalShardRate = replicaRates.stream()
- .map(r -> {
- String elapsedKey = r.getCollection() + "." + r.getCoreName();
- if ((Double)r.get(AutoScalingParams.RATE) > aboveRate) {
- if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
- hotReplicas.add(r);
- }
- } else if ((Double)r.get(AutoScalingParams.RATE) < belowRate) {
- if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
- coldReplicas.add(r);
- }
- } else {
- // no violation - clear waitForElapsed
- lastReplicaEvent.remove(elapsedKey);
- }
- return r;
- })
- .mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum();
- // calculate average shard rate over all searchable replicas (see SOLR-12470)
- double shardRate = totalShardRate / searchableReplicationFactors.get(coll).get(sh).doubleValue();
- String elapsedKey = coll + "." + sh;
- log.debug("-- {}: totalShardRate={}, shardRate={}", elapsedKey, totalShardRate, shardRate);
- if ((collections.isEmpty() || collections.contains(coll)) &&
- (shard.equals(Policy.ANY) || shard.equals(sh))) {
- if (shardRate > aboveRate) {
- if (waitForElapsed(elapsedKey, now, lastShardEvent)) {
- hotShards.computeIfAbsent(coll, s -> new HashMap<>()).put(sh, shardRate);
- }
- } else if (shardRate < belowRate) {
- if (waitForElapsed(elapsedKey, now, lastShardEvent)) {
- coldShards.computeIfAbsent(coll, s -> new HashMap<>()).put(sh, shardRate);
- log.debug("-- coldShard waitFor elapsed {}", elapsedKey);
- } else {
- if (log.isDebugEnabled()) {
- Long lastTime = lastShardEvent.computeIfAbsent(elapsedKey, s -> now);
- long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
- if (log.isDebugEnabled()) {
- log.debug("-- waitFor didn't elapse for {}, waitFor={}, elapsed={}", elapsedKey, getWaitForSecond(), elapsed);
- }
- }
- }
- } else {
- // no violation - clear waitForElapsed
- lastShardEvent.remove(elapsedKey);
- }
- }
- });
- });
-
- Map<String, Double> hotCollections = new HashMap<>();
- Map<String, Double> coldCollections = new HashMap<>();
- collectionRates.forEach((coll, shardRates) -> {
- double total = shardRates.entrySet().stream()
- .mapToDouble(e -> e.getValue().stream()
- .mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum()).sum();
- if (collections.isEmpty() || collections.contains(coll)) {
- if (total > aboveRate) {
- if (waitForElapsed(coll, now, lastCollectionEvent)) {
- hotCollections.put(coll, total);
- }
- } else if (total < belowRate) {
- if (waitForElapsed(coll, now, lastCollectionEvent)) {
- coldCollections.put(coll, total);
- }
- } else {
- // no violation - clear waitForElapsed
- lastCollectionEvent.remove(coll);
- }
- }
- });
-
- if (hotCollections.isEmpty() &&
- hotShards.isEmpty() &&
- hotReplicas.isEmpty() &&
- hotNodes.isEmpty() &&
- coldCollections.isEmpty() &&
- coldShards.isEmpty() &&
- coldReplicas.isEmpty() &&
- coldNodes.isEmpty()) {
- return;
- }
-
- // generate event
-
- // find the earliest time when a condition was exceeded
- final AtomicLong eventTime = new AtomicLong(now);
- hotCollections.forEach((c, r) -> {
- long time = lastCollectionEvent.get(c);
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- coldCollections.forEach((c, r) -> {
- long time = lastCollectionEvent.get(c);
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- hotShards.forEach((c, shards) -> {
- shards.forEach((s, r) -> {
- long time = lastShardEvent.get(c + "." + s);
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- });
- coldShards.forEach((c, shards) -> {
- shards.forEach((s, r) -> {
- long time = lastShardEvent.get(c + "." + s);
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- });
- hotReplicas.forEach(r -> {
- long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- coldReplicas.forEach(r -> {
- long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- hotNodes.forEach((n, r) -> {
- long time = lastNodeEvent.get(n);
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
- coldNodes.forEach((n, r) -> {
- long time = lastNodeEvent.get(n);
- if (eventTime.get() > time) {
- eventTime.set(time);
- }
- });
-
- final List<TriggerEvent.Op> ops = new ArrayList<>();
- final Set<String> violations = new HashSet<>();
-
- calculateHotOps(ops, violations, searchableReplicationFactors, hotNodes, hotCollections, hotShards, hotReplicas);
- calculateColdOps(ops, violations, clusterState, searchableReplicationFactors, coldNodes, coldCollections, coldShards, coldReplicas);
-
- if (ops.isEmpty()) {
- return;
- }
-
- if (processor.process(new SearchRateEvent(getName(), eventTime.get(), ops,
- hotNodes, hotCollections, hotShards, hotReplicas,
- coldNodes, coldCollections, coldShards, coldReplicas, violations))) {
- // update lastEvent times
- hotNodes.keySet().forEach(node -> lastNodeEvent.put(node, now));
- coldNodes.keySet().forEach(node -> lastNodeEvent.put(node, now));
- hotCollections.keySet().forEach(coll -> lastCollectionEvent.put(coll, now));
- coldCollections.keySet().forEach(coll -> lastCollectionEvent.put(coll, now));
- hotShards.entrySet().forEach(e -> e.getValue()
- .forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
- coldShards.entrySet().forEach(e -> e.getValue()
- .forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
- hotReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
- coldReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
- }
- }
-
- private void calculateHotOps(List<TriggerEvent.Op> ops,
- Set<String> violations,
- Map<String, Map<String, AtomicInteger>> searchableReplicationFactors,
- Map<String, Double> hotNodes,
- Map<String, Double> hotCollections,
- Map<String, Map<String, Double>> hotShards,
- List<Replica> hotReplicas) {
- // calculate the number of replicas to add to each hot shard, based on how much the rate was
- // exceeded - but within limits.
-
- // first resolve a situation when only a node is hot but no collection / shard is hot
- // TODO: eventually we may want to commission a new node
- if (!hotNodes.isEmpty()) {
- if (hotShards.isEmpty() && hotCollections.isEmpty()) {
- // move replicas around
- if (aboveNodeOp != null) {
- hotNodes.forEach((n, r) -> {
- ops.add(new TriggerEvent.Op(aboveNodeOp, Suggester.Hint.SRC_NODE, n));
- violations.add(HOT_NODES);
- });
- }
- } else {
- // ignore - hot shards will result in changes that will change hot node status anyway
- }
- }
- // add replicas
- Map<String, Map<String, List<Pair<String, String>>>> hints = new HashMap<>();
-
- // HOT COLLECTIONS
- // currently we don't do anything for hot collections. Theoretically we could add
- // 1 replica more to each shard, based on how close to the threshold each shard is
- // but it's probably better to wait for a shard to become hot and be more precise.
-
- // HOT SHARDS
-
- hotShards.forEach((coll, shards) -> shards.forEach((s, r) -> {
- List<Pair<String, String>> perShard = hints
- .computeIfAbsent(coll, c -> new HashMap<>())
- .computeIfAbsent(s, sh -> new ArrayList<>());
- addReplicaHints(coll, s, r, searchableReplicationFactors.get(coll).get(s).get(), perShard);
- violations.add(HOT_SHARDS);
- }));
-
- // HOT REPLICAS
- // Hot replicas (while their shards are not hot) may be caused by
- // dumb clients that use direct replica URLs - this is beyond our control
- // so ignore them.
-
- hints.values().forEach(m -> m.values().forEach(lst -> lst.forEach(p -> {
- ops.add(new TriggerEvent.Op(aboveOp, Suggester.Hint.COLL_SHARD, p));
- })));
-
- }
-
- /**
- * This method implements a primitive form of proportional controller with a limiter.
- */
- @SuppressWarnings({"unchecked", "rawtypes"})
- private void addReplicaHints(String collection, String shard, double r, int replicationFactor, List<Pair<String, String>> hints) {
- int numReplicas = (int)Math.round((r - aboveRate) / (double) replicationFactor);
- // in one event add at least 1 replica
- if (numReplicas < 1) {
- numReplicas = 1;
- }
- // ... and at most maxOps replicas
- if (numReplicas > maxOps) {
- numReplicas = maxOps;
- }
- for (int i = 0; i < numReplicas; i++) {
- hints.add(new Pair(collection, shard));
- }
- }
-
- private void calculateColdOps(List<TriggerEvent.Op> ops,
- Set<String> violations,
- ClusterState clusterState,
- Map<String, Map<String, AtomicInteger>> searchableReplicationFactors,
- Map<String, Double> coldNodes,
- Map<String, Double> coldCollections,
- Map<String, Map<String, Double>> coldShards,
- List<Replica> coldReplicas) {
- // COLD COLLECTIONS
- // Probably can't do anything reasonable about whole cold collections
- // because they may be needed even if not used.
-
- // COLD SHARDS & COLD REPLICAS:
- // We remove cold replicas only from cold shards, otherwise we are susceptible to uneven
- // replica routing (which is beyond our control).
- // If we removed replicas from non-cold shards we could accidentally bring that shard into
- // the hot range, which would result in adding replica, and that replica could again stay cold due to
- // the same routing issue, which then would lead to removing that replica, etc, etc...
-
- // Remove cold replicas but only when there's at least a minimum number of searchable
- // replicas still available (additional non-searchable replicas may exist, too)
- // NOTE: do this before adding ops for DELETENODE because we don't want to attempt
- // deleting replicas that have been already moved elsewhere
- Map<String, Map<String, List<Replica>>> byCollectionByShard = new HashMap<>();
- coldReplicas.forEach(ri -> {
- byCollectionByShard.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
- .computeIfAbsent(ri.getShard(), s -> new ArrayList<>())
- .add(ri);
- });
- coldShards.forEach((coll, perShard) -> {
- perShard.forEach((shard, rate) -> {
- List<Replica> replicas = byCollectionByShard
- .getOrDefault(coll, Collections.emptyMap())
- .getOrDefault(shard, Collections.emptyList());
- if (replicas.isEmpty()) {
- return;
- }
- // only delete if there's at least minRF searchable replicas left
- int rf = searchableReplicationFactors.get(coll).get(shard).get();
- // assume first that we only really need a leader and we may be
- // allowed to remove other replicas
- int minRF = 1;
- // but check the official RF and don't go below that
- Integer RF = clusterState.getCollection(coll).getReplicationFactor();
- if (RF != null) {
- minRF = RF;
- }
- // unless minReplicas is set explicitly
- if (minReplicas != null) {
- minRF = minReplicas;
- }
- if (minRF < 1) {
- minRF = 1;
- }
- if (rf > minRF) {
- // delete at most maxOps replicas at a time
- AtomicInteger limit = new AtomicInteger(Math.min(maxOps, rf - minRF));
- replicas.forEach(ri -> {
- if (limit.get() == 0) {
- return;
- }
- // don't delete a leader
- if (ri.getBool(ZkStateReader.LEADER_PROP, false)) {
- return;
- }
- TriggerEvent.Op op = new TriggerEvent.Op(belowOp,
- Suggester.Hint.COLL_SHARD, new Pair<>(ri.getCollection(), ri.getShard()));
- op.addHint(Suggester.Hint.REPLICA, ri.getName());
- ops.add(op);
- violations.add(COLD_SHARDS);
- limit.decrementAndGet();
- });
- }
- });
- });
-
- // COLD NODES:
- // Unlike the case of hot nodes, if a node is cold then any monitored
- // collections / shards / replicas located on that node are cold, too.
- // HOWEVER, we check only replicas from selected collections / shards,
- // so deleting a cold node is dangerous because it may interfere with these
- // non-monitored resources - this is the reason the default belowNodeOp is null / ignored.
- //
- // Also, note that due to the way activity is measured only nodes that contain any
- // monitored resources are considered - there may be cold nodes in the cluster that don't
- // belong to the monitored collections and they will be ignored.
- if (belowNodeOp != null) {
- coldNodes.forEach((node, rate) -> {
- ops.add(new TriggerEvent.Op(belowNodeOp, Suggester.Hint.SRC_NODE, node));
- violations.add(COLD_NODES);
- });
- }
-
-
- }
-
- private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
- Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
- long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
- if (log.isTraceEnabled()) {
- log.trace("name={}, lastTime={}, elapsed={}, waitFor={}", name, lastTime, elapsed, getWaitForSecond());
- }
- if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
- return false;
- }
- return true;
- }
-
- public static class SearchRateEvent extends TriggerEvent {
- public SearchRateEvent(String source, long eventTime, List<Op> ops,
- Map<String, Double> hotNodes,
- Map<String, Double> hotCollections,
- Map<String, Map<String, Double>> hotShards,
- List<Replica> hotReplicas,
- Map<String, Double> coldNodes,
- Map<String, Double> coldCollections,
- Map<String, Map<String, Double>> coldShards,
- List<Replica> coldReplicas,
- Set<String> violations) {
- super(TriggerEventType.SEARCHRATE, source, eventTime, null);
- properties.put(TriggerEvent.REQUESTED_OPS, ops);
- properties.put(HOT_NODES, hotNodes);
- properties.put(HOT_COLLECTIONS, hotCollections);
- properties.put(HOT_SHARDS, hotShards);
- properties.put(HOT_REPLICAS, hotReplicas);
- properties.put(COLD_NODES, coldNodes);
- properties.put(COLD_COLLECTIONS, coldCollections);
- properties.put(COLD_SHARDS, coldShards);
- properties.put(COLD_REPLICAS, coldReplicas);
- properties.put(VIOLATION_PROP, violations);
- }
- }
-}
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
deleted file mode 100644
index b841478..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
+++ /dev/null
@@ -1,221 +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.PrintWriter;
-import java.io.StringWriter;
-import java.lang.invoke.MethodHandles;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-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.request.UpdateRequest;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.params.CollectionAdminParams;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.SolrParams;
-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.IdUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This listener saves events to the {@link CollectionAdminParams#SYSTEM_COLL} collection.
- * <p>Configuration properties:</p>
- * <ul>
- * <li>collection - optional string, specifies what collection should be used for storing events. Default value
- * is {@link CollectionAdminParams#SYSTEM_COLL}.</li>
- * </ul>
- */
-public class SystemLogListener extends TriggerListenerBase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public static final String SOURCE_FIELD = "source_s";
- public static final String EVENT_SOURCE_FIELD = "event.source_s";
- public static final String EVENT_TYPE_FIELD = "event.type_s";
- public static final String STAGE_FIELD = "stage_s";
- public static final String ACTION_FIELD = "action_s";
- public static final String MESSAGE_FIELD = "message_t";
- public static final String BEFORE_ACTIONS_FIELD = "before.actions_ss";
- public static final String AFTER_ACTIONS_FIELD = "after.actions_ss";
- public static final String COLLECTIONS_FIELD = "collections_ss";
- public static final String SOURCE = SystemLogListener.class.getSimpleName();
- public static final String DOC_TYPE = "autoscaling_event";
-
- private String collection = CollectionAdminParams.SYSTEM_COLL;
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
- super.configure(loader, cloudManager, config);
- collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
- }
-
- @Override
- @SuppressWarnings({"unchecked", "rawtypes"})
- public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
- Throwable error, String message) throws Exception {
- try {
- ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
- DocCollection coll = clusterState.getCollectionOrNull(collection);
- if (coll == null) {
- log.debug("Collection {} missing, skip sending event {}", collection, event);
- return;
- }
- SolrInputDocument doc = new SolrInputDocument();
- doc.addField(CommonParams.TYPE, DOC_TYPE);
- doc.addField(SOURCE_FIELD, SOURCE);
- doc.addField("id", IdUtils.timeRandomId());
- doc.addField("event.id_s", event.getId());
- doc.addField(EVENT_TYPE_FIELD, event.getEventType().toString());
- doc.addField(EVENT_SOURCE_FIELD, event.getSource());
- doc.addField("event.time_l", event.getEventTime());
- doc.addField("timestamp", new Date());
- addMap("event.property.", doc, event.getProperties());
- doc.addField(STAGE_FIELD, stage.toString());
- if (actionName != null) {
- doc.addField(ACTION_FIELD, actionName);
- }
- if (message != null) {
- doc.addField(MESSAGE_FIELD, message);
- }
- addError(doc, error);
- // add JSON versions of event and context
- String eventJson = Utils.toJSONString(event);
- doc.addField("event_str", eventJson);
- if (context != null) {
- // capture specifics of operations after compute_plan action
- addOperations(doc, (List<SolrRequest>)context.getProperties().get("operations"));
- // capture specifics of responses after execute_plan action
- addResponses(doc, (List<NamedList<Object>>)context.getProperties().get("responses"));
- addActions(BEFORE_ACTIONS_FIELD, doc, (List<String>)context.getProperties().get(TriggerEventProcessorStage.BEFORE_ACTION.toString()));
- addActions(AFTER_ACTIONS_FIELD, doc, (List<String>)context.getProperties().get(TriggerEventProcessorStage.AFTER_ACTION.toString()));
- String contextJson = Utils.toJSONString(context);
- doc.addField("context_str", contextJson);
- }
- UpdateRequest req = new UpdateRequest();
- req.add(doc);
- req.setParam(CollectionAdminParams.COLLECTION, collection);
- cloudManager.request(req);
- } catch (Exception e) {
- if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
- // relatively benign but log this - collection still existed when we started
- log.info("Collection {} missing, skip sending event {}", collection, event);
- } else {
- log.warn("Exception sending event. Collection: {}, event: {}, exception: {}", collection, event, e);
- }
- }
- }
-
- private void addActions(String field, SolrInputDocument doc, List<String> actions) {
- if (actions == null) {
- return;
- }
- actions.forEach(a -> doc.addField(field, a));
- }
-
- private void addMap(String prefix, SolrInputDocument doc, Map<String, Object> map) {
- map.forEach((k, v) -> {
- if (v instanceof Collection) {
- for (Object o : (Collection)v) {
- doc.addField(prefix + k + "_ss", String.valueOf(o));
- }
- } else {
- doc.addField(prefix + k + "_ss", String.valueOf(v));
- }
- });
- }
-
- @SuppressWarnings({"rawtypes"})
- private void addOperations(SolrInputDocument doc, List<SolrRequest> operations) {
- if (operations == null || operations.isEmpty()) {
- return;
- }
- Set<String> collections = new HashSet<>();
- for (SolrRequest req : operations) {
- SolrParams params = req.getParams();
- if (params == null) {
- continue;
- }
- if (params.get(CollectionAdminParams.COLLECTION) != null) {
- collections.add(params.get(CollectionAdminParams.COLLECTION));
- }
- // build a whitespace-separated param string
- StringJoiner paramJoiner = new StringJoiner(" ");
- paramJoiner.setEmptyValue("");
- for (Iterator<String> it = params.getParameterNamesIterator(); it.hasNext(); ) {
- final String name = it.next();
- final String [] values = params.getParams(name);
- for (String value : values) {
- paramJoiner.add(name + "=" + value);
- }
- }
- String paramString = paramJoiner.toString();
- if (!paramString.isEmpty()) {
- doc.addField("operations.params_ts", paramString);
- }
- }
- if (!collections.isEmpty()) {
- doc.addField(COLLECTIONS_FIELD, collections);
- }
- }
-
- private void addResponses(SolrInputDocument doc, List<NamedList<Object>> responses) {
- if (responses == null || responses.isEmpty()) {
- return;
- }
- for (NamedList<Object> rsp : responses) {
- Object o = rsp.get("success");
- if (o != null) {
- doc.addField("responses_ts", "success " + o);
- } else {
- o = rsp.get("failure");
- if (o != null) {
- doc.addField("responses_ts", "failure " + o);
- } else { // something else
- doc.addField("responses_ts", Utils.toJSONString(rsp));
- }
- }
- }
- }
-
- private void addError(SolrInputDocument doc, Throwable error) {
- if (error == null) {
- return;
- }
- StringWriter sw = new StringWriter();
- PrintWriter pw = new PrintWriter(sw);
- error.printStackTrace(pw);
- pw.flush(); pw.close();
- doc.addField("error.message_t", error.getMessage());
- doc.addField("error.details_t", sw.toString());
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java
deleted file mode 100644
index b873ee6..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java
+++ /dev/null
@@ -1,51 +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.util.Map;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.core.SolrResourceLoader;
-
-/**
- * Interface for actions performed in response to a trigger being activated
- */
-public interface TriggerAction extends Closeable {
-
- /**
- * Called when action is created but before it's initialized and used.
- * This method should also verify that the configuration parameters are correct.
- * It may be called multiple times.
- * @param loader loader to use for instantiating sub-components
- * @param cloudManager current instance of SolrCloudManager
- * @param properties configuration properties
- * @throws TriggerValidationException contains details of invalid configuration parameters.
- */
- void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException;
-
- /**
- * Called before an action is first used. Any heavy object creation or initialization should
- * be done in this method instead of the constructor or {@link #configure(SolrResourceLoader, SolrCloudManager, Map)} method.
- */
- void init() throws Exception;
-
- String getName();
-
- void process(TriggerEvent event, ActionContext context) throws Exception;
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java
deleted file mode 100644
index 7a9f34b..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java
+++ /dev/null
@@ -1,87 +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.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.core.SolrResourceLoader;
-
-/**
- * Base class for {@link TriggerAction} implementations.
- */
-public abstract class TriggerActionBase implements TriggerAction {
-
- protected Map<String, Object> properties = new HashMap<>();
- protected SolrResourceLoader loader;
- protected SolrCloudManager cloudManager;
- /**
- * Set of valid property names. Subclasses may add to this set
- * using {@link TriggerUtils#validProperties(Set, String...)}
- */
- protected final Set<String> validProperties = new HashSet<>();
- /**
- * Set of required property names. Subclasses may add to this set
- * using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
- * (required properties are also valid properties).
- */
- protected final Set<String> requiredProperties = new HashSet<>();
-
- protected TriggerActionBase() {
- // not strictly needed here because they are already checked during instantiation
- TriggerUtils.validProperties(validProperties, "name", "class");
- }
-
- @Override
- public String getName() {
- String name = (String) properties.get("name");
- if (name != null) {
- return name;
- } else {
- return getClass().getSimpleName();
- }
- }
-
- @Override
- public void close() throws IOException {
-
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
- this.loader = loader;
- this.cloudManager = cloudManager;
- if (properties != null) {
- this.properties.putAll(properties);
- }
- // validate the config
- Map<String, String> results = new HashMap<>();
- TriggerUtils.checkProperties(this.properties, results, requiredProperties, validProperties);
- if (!results.isEmpty()) {
- throw new TriggerValidationException(getName(), results);
- }
- }
-
- @Override
- public void init() throws Exception {
-
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionException.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionException.java
deleted file mode 100644
index 624ce68..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionException.java
+++ /dev/null
@@ -1,33 +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;
-
-/**
- * Trigger action-specific exception.
- */
-public class TriggerActionException extends Exception {
-
- public final String triggerName;
- public final String actionName;
-
- public TriggerActionException(String triggerName, String actionName, String message, Throwable cause) {
- super(message, cause);
- this.triggerName = triggerName;
- this.actionName = actionName;
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
deleted file mode 100644
index d045f6a..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
+++ /dev/null
@@ -1,300 +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.Objects;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.lucene.util.IOUtils;
-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.DistribStateManager;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
-import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Base class for {@link org.apache.solr.cloud.autoscaling.AutoScaling.Trigger} implementations.
- * It handles state snapshot / restore in ZK.
- */
-public abstract class TriggerBase implements AutoScaling.Trigger {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- protected final String name;
- protected SolrCloudManager cloudManager;
- protected SolrResourceLoader loader;
- protected DistribStateManager stateManager;
- protected final Map<String, Object> properties = new HashMap<>();
- /**
- * Set of valid property names. Subclasses may add to this set
- * using {@link TriggerUtils#validProperties(Set, String...)}
- */
- protected final Set<String> validProperties = new HashSet<>();
- /**
- * Set of required property names. Subclasses may add to this set
- * using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
- * (required properties are also valid properties).
- */
- protected final Set<String> requiredProperties = new HashSet<>();
- protected final TriggerEventType eventType;
- protected int waitForSecond;
- protected Map<String,Object> lastState;
- protected final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef = new AtomicReference<>();
- protected List<TriggerAction> actions;
- protected boolean enabled;
- protected boolean isClosed;
-
-
- protected TriggerBase(TriggerEventType eventType, String name) {
- this.eventType = eventType;
- this.name = name;
-
- // subclasses may further modify this set to include other supported properties
- TriggerUtils.validProperties(validProperties, "name", "class", "event", "enabled", "waitFor", "actions");
- }
-
- /**
- * Return a set of valid property names supported by this trigger.
- */
- public final Set<String> getValidProperties() {
- return Collections.unmodifiableSet(this.validProperties);
- }
-
- /**
- * Return a set of required property names supported by this trigger.
- */
- public final Set<String> getRequiredProperties() {
- return Collections.unmodifiableSet(this.requiredProperties);
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
- this.cloudManager = cloudManager;
- this.loader = loader;
- this.stateManager = cloudManager.getDistribStateManager();
- if (properties != null) {
- this.properties.putAll(properties);
- }
- this.enabled = Boolean.parseBoolean(String.valueOf(this.properties.getOrDefault("enabled", "true")));
- this.waitForSecond = ((Number) this.properties.getOrDefault("waitFor", -1L)).intValue();
- @SuppressWarnings({"unchecked"})
- List<Map<String, Object>> o = (List<Map<String, Object>>) properties.get("actions");
- if (o != null && !o.isEmpty()) {
- actions = new ArrayList<>(3);
- for (Map<String, Object> map : o) {
- TriggerAction action = null;
- try {
- action = loader.newInstance((String)map.get("class"), TriggerAction.class);
- } catch (Exception e) {
- throw new TriggerValidationException("action", "exception creating action " + map + ": " + e.toString());
- }
- action.configure(loader, cloudManager, map);
- actions.add(action);
- }
- } else {
- actions = Collections.emptyList();
- }
-
-
- Map<String, String> results = new HashMap<>();
- TriggerUtils.checkProperties(this.properties, results, requiredProperties, validProperties);
- if (!results.isEmpty()) {
- throw new TriggerValidationException(name, results);
- }
- }
-
- @Override
- public void init() throws Exception {
- try {
- if (!stateManager.hasData(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH)) {
- stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
- }
- } catch (AlreadyExistsException e) {
- // ignore
- } catch (InterruptedException | KeeperException | IOException e) {
- log.warn("Exception checking ZK path {}", ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, e);
- throw e;
- }
- for (TriggerAction action : actions) {
- action.init();
- }
- }
-
- @Override
- public void setProcessor(AutoScaling.TriggerEventProcessor processor) {
- processorRef.set(processor);
- }
-
- @Override
- public AutoScaling.TriggerEventProcessor getProcessor() {
- return processorRef.get();
- }
-
- @Override
- public String getName() {
- return name;
- }
-
- @Override
- public TriggerEventType getEventType() {
- return eventType;
- }
-
- @Override
- public boolean isEnabled() {
- return enabled;
- }
-
- @Override
- public int getWaitForSecond() {
- return waitForSecond;
- }
-
- @Override
- public Map<String, Object> getProperties() {
- return properties;
- }
-
- @Override
- public List<TriggerAction> getActions() {
- return actions;
- }
-
- @Override
- public boolean isClosed() {
- synchronized (this) {
- return isClosed;
- }
- }
-
- @Override
- public void close() throws IOException {
- synchronized (this) {
- isClosed = true;
- IOUtils.closeWhileHandlingException(actions);
- }
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(name, properties);
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj == null) {
- return false;
- }
- if (obj.getClass().equals(this.getClass())) {
- TriggerBase that = (TriggerBase) obj;
- return this.name.equals(that.name)
- && this.properties.equals(that.properties);
- }
- return false;
- }
-
- /**
- * Prepare and return internal state of this trigger in a format suitable for persisting in ZK.
- * @return map of internal state properties. Note: values must be supported by {@link Utils#toJSON(Object)}.
- */
- protected abstract Map<String,Object> getState();
-
- /**
- * Restore internal state of this trigger from properties retrieved from ZK.
- * @param state never null but may be empty.
- */
- protected abstract void setState(Map<String,Object> state);
-
- /**
- * Returns an immutable deep copy of this trigger's state, suitible for saving.
- * This method is public only for tests that wish to do grey-box introspection
- *
- * @see #getState
- * @lucene.internal
- */
- @SuppressWarnings({"unchecked"})
- public Map<String,Object> deepCopyState() {
- return Utils.getDeepCopy(getState(), 10, false, true);
- }
-
- @Override
- public void saveState() {
- Map<String,Object> state = deepCopyState();
- if (lastState != null && lastState.equals(state)) {
- // skip saving if identical
- return;
- }
- byte[] data = Utils.toJSON(state);
- String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName();
- try {
- if (stateManager.hasData(path)) {
- // update
- stateManager.setData(path, data, -1);
- } else {
- // create
- stateManager.createData(path, data, CreateMode.PERSISTENT);
- }
- lastState = state;
- } catch (AlreadyExistsException e) {
-
- } catch (InterruptedException | BadVersionException | IOException | KeeperException e) {
- log.warn("Exception updating trigger state '{}'", path, e);
- }
- }
-
- @Override
- @SuppressWarnings({"unchecked"})
- public void restoreState() {
- byte[] data = null;
- String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName();
- try {
- if (stateManager.hasData(path)) {
- VersionedData versionedData = stateManager.getData(path);
- data = versionedData.getData();
- }
- } catch (AlreadyClosedException e) {
-
- } catch (Exception e) {
- log.warn("Exception getting trigger state '{}'", path, e);
- }
- if (data != null) {
- Map<String, Object> restoredState = (Map<String, Object>)Utils.fromJSON(data);
- // make sure lastState is sorted
- restoredState = Utils.getDeepCopy(restoredState, 10, false, true);
- setState(restoredState);
- lastState = restoredState;
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java
deleted file mode 100644
index 91482e5..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java
+++ /dev/null
@@ -1,315 +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.Collection;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.common.MapWriter;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.util.IdUtils;
-
-/**
- * Trigger event.
- */
-public class TriggerEvent implements MapWriter {
- public static final String IGNORED = "ignored";
- public static final String COOLDOWN = "cooldown";
- public static final String REPLAYING = "replaying";
- public static final String NODE_NAMES = "nodeNames";
- public static final String EVENT_TIMES = "eventTimes";
- public static final String REQUESTED_OPS = "requestedOps";
- public static final String UNSUPPORTED_OPS = "unsupportedOps";
-
- public static final class Op implements MapWriter {
- private final CollectionParams.CollectionAction action;
- private final EnumMap<Suggester.Hint, Object> hints = new EnumMap<>(Suggester.Hint.class);
-
- public Op(CollectionParams.CollectionAction action) {
- this.action = action;
- }
-
- public Op(CollectionParams.CollectionAction action, Suggester.Hint hint, Object hintValue) {
- this.action = action;
- addHint(hint, hintValue);
- }
-
- @SuppressWarnings({"unchecked"})
- public void addHint(Suggester.Hint hint, Object value) {
- hint.validator.accept(value);
- if (hint.multiValued) {
- Collection<?> values = value instanceof Collection ? (Collection) value : Collections.singletonList(value);
- ((Set) hints.computeIfAbsent(hint, h -> new LinkedHashSet<>())).addAll(values);
- } else if (value instanceof Map) {
- hints.put(hint, value);
- } else {
- hints.put(hint, value == null ? null : String.valueOf(value));
- }
- }
-
- public CollectionParams.CollectionAction getAction() {
- return action;
- }
-
- public EnumMap<Suggester.Hint, Object> getHints() {
- return hints;
- }
-
- @Override
- public void writeMap(EntryWriter ew) throws IOException {
- ew.put("action", action);
- ew.put("hints", hints);
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- public static Op fromMap(Map<String, Object> map) {
- if (!map.containsKey("action")) {
- return null;
- }
- CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(String.valueOf(map.get("action")));
- if (action == null) {
- return null;
- }
- Op op = new Op(action);
- Map<Object, Object> hints = (Map<Object, Object>)map.get("hints");
- if (hints != null && !hints.isEmpty()) {
- hints.forEach((k, v) -> {
- Suggester.Hint h = Suggester.Hint.get(k.toString());
- if (h == null) {
- return;
- }
- if (!(v instanceof Collection)) {
- v = Collections.singletonList(v);
- }
- ((Collection)v).forEach(vv -> {
- if (vv instanceof Map) {
- // maybe it's a Pair?
- Map<String, Object> m = (Map<String, Object>)vv;
- if (m.containsKey("first") && m.containsKey("second")) {
- Pair p = Pair.parse(m);
- if (p != null) {
- op.addHint(h, p);
- return;
- }
- }
- }
- op.addHint(h, vv);
- });
- });
- }
- return op;
- }
-
- @Override
- public String toString() {
- return "Op{" +
- "action=" + action +
- ", hints=" + hints +
- '}';
- }
- }
-
- protected final String id;
- protected final String source;
- protected final long eventTime;
- protected final TriggerEventType eventType;
- protected final Map<String, Object> properties = new HashMap<>();
- protected final boolean ignored;
-
- public TriggerEvent(TriggerEventType eventType, String source, long eventTime,
- Map<String, Object> properties) {
- this(IdUtils.timeRandomId(eventTime), eventType, source, eventTime, properties, false);
- }
-
- public TriggerEvent(TriggerEventType eventType, String source, long eventTime,
- Map<String, Object> properties, boolean ignored) {
- this(IdUtils.timeRandomId(eventTime), eventType, source, eventTime, properties, ignored);
- }
-
- public TriggerEvent(String id, TriggerEventType eventType, String source, long eventTime,
- Map<String, Object> properties) {
- this(id, eventType, source, eventTime, properties, false);
- }
-
- public TriggerEvent(String id, TriggerEventType eventType, String source, long eventTime,
- Map<String, Object> properties, boolean ignored) {
- this.id = id;
- this.eventType = eventType;
- this.source = source;
- this.eventTime = eventTime;
- if (properties != null) {
- this.properties.putAll(properties);
- }
- this.ignored = ignored;
- }
-
- /**
- * Unique event id.
- */
- public String getId() {
- return id;
- }
-
- /**
- * Name of the trigger that fired the event.
- */
- public String getSource() {
- return source;
- }
-
- /**
- * Timestamp of the actual event, in nanoseconds.
- * NOTE: this is NOT the timestamp when the event was fired - events may be fired
- * much later than the actual condition that generated the event, due to the "waitFor" limit.
- */
- public long getEventTime() {
- return eventTime;
- }
-
- /**
- * Get event properties (modifiable).
- */
- public Map<String, Object> getProperties() {
- return properties;
- }
-
- /**
- * Get a named event property or null if missing.
- */
- public Object getProperty(String name) {
- return properties.get(name);
- }
-
- /**
- * Get a named event property or default value if missing.
- */
- public Object getProperty(String name, Object defaultValue) {
- Object v = properties.get(name);
- if (v == null) {
- return defaultValue;
- } else {
- return v;
- }
- }
-
- /**
- * Event type.
- */
- public TriggerEventType getEventType() {
- return eventType;
- }
-
- public boolean isIgnored() {
- return ignored;
- }
-
- /**
- * Set event properties.
- *
- * @param properties may be null. A shallow copy of this parameter is used.
- */
- public void setProperties(Map<String, Object> properties) {
- this.properties.clear();
- if (properties != null) {
- this.properties.putAll(properties);
- }
- }
-
- @Override
- public void writeMap(EntryWriter ew) throws IOException {
- ew.put("id", id);
- ew.put("source", source);
- ew.put("eventTime", eventTime);
- ew.put("eventType", eventType.toString());
- ew.put("properties", properties);
- if (ignored) {
- ew.put("ignored", true);
- }
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- TriggerEvent that = (TriggerEvent) o;
-
- if (eventTime != that.eventTime) return false;
- if (!id.equals(that.id)) return false;
- if (!source.equals(that.source)) return false;
- if (eventType != that.eventType) return false;
- if (ignored != that.ignored) return false;
- return properties.equals(that.properties);
- }
-
- @Override
- public int hashCode() {
- int result = id.hashCode();
- result = 31 * result + source.hashCode();
- result = 31 * result + (int) (eventTime ^ (eventTime >>> 32));
- result = 31 * result + eventType.hashCode();
- result = 31 * result + properties.hashCode();
- result = 31 * result + Boolean.hashCode(ignored);
- return result;
- }
-
- @Override
- public String toString() {
- return Utils.toJSONString(this);
- }
-
- @SuppressWarnings({"unchecked"})
- public static TriggerEvent fromMap(Map<String, Object> map) {
- String id = (String)map.get("id");
- String source = (String)map.get("source");
- long eventTime = ((Number)map.get("eventTime")).longValue();
- TriggerEventType eventType = TriggerEventType.valueOf((String)map.get("eventType"));
- Map<String, Object> properties = (Map<String, Object>)map.get("properties");
- // properly deserialize some well-known complex properties
- fixOps(TriggerEvent.REQUESTED_OPS, properties);
- fixOps(TriggerEvent.UNSUPPORTED_OPS, properties);
- TriggerEvent res = new TriggerEvent(id, eventType, source, eventTime, properties);
- return res;
- }
-
- @SuppressWarnings({"unchecked"})
- public static void fixOps(String type, Map<String, Object> properties) {
- List<Object> ops = (List<Object>)properties.get(type);
- if (ops != null && !ops.isEmpty()) {
- for (int i = 0; i < ops.size(); i++) {
- Object o = ops.get(i);
- if (o instanceof Map) {
- TriggerEvent.Op op = TriggerEvent.Op.fromMap((Map)o);
- if (op != null) {
- ops.set(i, op);
- }
- }
- }
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
deleted file mode 100644
index ec41495..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
+++ /dev/null
@@ -1,121 +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.nio.charset.StandardCharsets;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.cloud.DistributedQueue;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.cloud.Stats;
-import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.common.util.TimeSource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- */
-public class TriggerEventQueue {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- public static final String ENQUEUE_TIME = "_enqueue_time_";
- public static final String DEQUEUE_TIME = "_dequeue_time_";
-
- private final String triggerName;
- private final TimeSource timeSource;
- private final DistributedQueue delegate;
-
- public TriggerEventQueue(SolrCloudManager cloudManager, String triggerName, Stats stats) throws IOException {
- // TODO: collect stats
- this.delegate = cloudManager.getDistributedQueueFactory().makeQueue(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName);
- this.triggerName = triggerName;
- this.timeSource = cloudManager.getTimeSource();
- }
-
- public boolean offerEvent(TriggerEvent event) {
- event.getProperties().put(ENQUEUE_TIME, timeSource.getTimeNs());
- try {
- byte[] data = Utils.toJSON(event);
- delegate.offer(data);
- return true;
- } catch (Exception e) {
- log.warn("Exception adding event {} to queue {}", event, triggerName, e);
- return false;
- }
- }
-
- public TriggerEvent peekEvent() {
- byte[] data;
- try {
- while ((data = delegate.peek()) != null) {
- if (data.length == 0) {
- log.warn("ignoring empty data...");
- continue;
- }
- try {
- @SuppressWarnings({"unchecked"})
- Map<String, Object> map = (Map<String, Object>) Utils.fromJSON(data);
- return fromMap(map);
- } catch (Exception e) {
- log.warn("Invalid event data, ignoring: {}", new String(data, StandardCharsets.UTF_8));
- continue;
- }
- }
- }
- catch (AlreadyClosedException e) {
-
- }
- catch (Exception e) {
- log.warn("Exception peeking queue of trigger {}", triggerName, e);
- }
- return null;
- }
-
- public TriggerEvent pollEvent() {
- byte[] data;
- try {
- while ((data = delegate.poll()) != null) {
- if (data.length == 0) {
- log.warn("ignoring empty data...");
- continue;
- }
- try {
- @SuppressWarnings({"unchecked"})
- Map<String, Object> map = (Map<String, Object>) Utils.fromJSON(data);
- return fromMap(map);
- } catch (Exception e) {
- log.warn("Invalid event data, ignoring: {}", new String(data, StandardCharsets.UTF_8));
- continue;
- }
- }
- } catch (Exception e) {
- log.warn("Exception polling queue of trigger {}", triggerName, e);
- }
- return null;
- }
-
- private TriggerEvent fromMap(Map<String, Object> map) {
- TriggerEvent res = TriggerEvent.fromMap(map);
- res.getProperties().put(DEQUEUE_TIME, timeSource.getTimeNs());
- return res;
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
deleted file mode 100644
index 234387f..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
+++ /dev/null
@@ -1,65 +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 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.core.SolrResourceLoader;
-
-/**
- * Implementations of this interface are notified of stages in event processing that they were
- * registered for. Note: instances may be closed and re-created on each auto-scaling config update.
- */
-public interface TriggerListener extends Closeable {
-
- /**
- * Called when listener is created but before it's initialized and used.
- * This method should also verify that the configuration parameters are correct.
- * It may be called multiple times.
- * @param loader loader to use for instantiating sub-components
- * @param cloudManager current instance of SolrCloudManager
- * @param config coniguration
- * @throws TriggerValidationException contains details of invalid configuration parameters.
- */
- void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException;
-
- /**
- * If this method returns false then the listener's {@link #onEvent(TriggerEvent, TriggerEventProcessorStage, String, ActionContext, Throwable, String)}
- * method should not be called.
- */
- boolean isEnabled();
-
- void init() throws Exception;
-
- AutoScalingConfig.TriggerListenerConfig getConfig();
-
- /**
- * This method is called when either a particular <code>stage</code> or
- * <code>actionName</code> is reached during event processing.
- * @param event current event being processed
- * @param stage {@link TriggerEventProcessorStage} that this listener was registered for, or null
- * @param actionName {@link TriggerAction} name that this listener was registered for, or null
- * @param context optional {@link ActionContext} when the processing stage is related to an action, or null
- * @param error optional {@link Throwable} error, or null
- * @param message optional message
- */
- void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
- Throwable error, String message) throws Exception;
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
deleted file mode 100644
index 7a323c7..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
+++ /dev/null
@@ -1,97 +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.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.core.SolrResourceLoader;
-
-/**
- * Base class for implementations of {@link TriggerListener}.
- */
-public abstract class TriggerListenerBase implements TriggerListener {
-
- protected AutoScalingConfig.TriggerListenerConfig config;
- protected SolrCloudManager cloudManager;
- protected SolrResourceLoader loader;
- protected boolean enabled;
- /**
- * Set of valid property names. Subclasses may add to this set
- * using {@link TriggerUtils#validProperties(Set, String...)}
- */
- protected final Set<String> validProperties = new HashSet<>();
- /**
- * Set of required property names. Subclasses may add to this set
- * using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
- * (required properties are also valid properties).
- */
- protected final Set<String> requiredProperties = new HashSet<>();
- /**
- * Subclasses can add to this set if they want to allow arbitrary properties that
- * start with one of valid prefixes.
- */
- protected final Set<String> validPropertyPrefixes = new HashSet<>();
-
- protected TriggerListenerBase() {
- TriggerUtils.requiredProperties(requiredProperties, validProperties, "trigger");
- TriggerUtils.validProperties(validProperties, "name", "class", "stage", "beforeAction", "afterAction", "enabled");
- }
-
- @Override
- public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
- this.loader = loader;
- this.cloudManager = cloudManager;
- this.config = config;
- this.enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
- // validate the config
- Map<String, String> results = new HashMap<>();
- // prepare a copy to treat the prefix-based properties
- Map<String, Object> propsToCheck = new HashMap<>(config.properties);
- propsToCheck.keySet().removeIf(k ->
- validPropertyPrefixes.stream().anyMatch(p -> k.startsWith(p)));
- TriggerUtils.checkProperties(propsToCheck, results, requiredProperties, validProperties);
- if (!results.isEmpty()) {
- throw new TriggerValidationException(config.name, results);
- }
- }
-
- @Override
- public AutoScalingConfig.TriggerListenerConfig getConfig() {
- return config;
- }
-
- @Override
- public boolean isEnabled() {
- return enabled;
- }
-
- @Override
- public void init() throws Exception {
-
- }
-
- @Override
- public void close() throws IOException {
-
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java
deleted file mode 100644
index cecd933..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java
+++ /dev/null
@@ -1,88 +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.Arrays;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- *
- */
-public class TriggerUtils {
- // validation helper methods
-
- public static void requiredProperties(Set<String> required, Set<String> valid, String... propertyNames) {
- required.addAll(Arrays.asList(propertyNames));
- valid.addAll(Arrays.asList(propertyNames));
- }
-
- public static void validProperties(Set<String> valid, String... propertyNames) {
- valid.addAll(Arrays.asList(propertyNames));
- }
-
- public static void checkProperties(Map<String, Object> properties, Map<String, String> results, Set<String> required, Set<String> valid) {
- checkValidPropertyNames(properties, results, valid);
- checkRequiredPropertyNames(properties, results, required);
- }
-
- public static void checkValidPropertyNames(Map<String, Object> properties, Map<String, String> results, Set<String> valid) {
- Set<String> currentNames = new HashSet<>(properties.keySet());
- currentNames.removeAll(valid);
- if (!currentNames.isEmpty()) {
- for (String name : currentNames) {
- results.put(name, "unknown property");
- }
- }
- }
-
- public static void checkRequiredPropertyNames(Map<String, Object> properties, Map<String, String> results, Set<String> required) {
- Set<String> requiredNames = new HashSet<>(required);
- requiredNames.removeAll(properties.keySet());
- if (!requiredNames.isEmpty()) {
- for (String name : requiredNames) {
- results.put(name, "missing required property");
- }
- }
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- public static void checkProperty(Map<String, Object> properties, Map<String, String> results, String name, boolean required, Class... acceptClasses) {
- Object value = properties.get(name);
- if (value == null) {
- if (required) {
- results.put(name, "missing required value");
- } else {
- return;
- }
- }
- if (acceptClasses == null || acceptClasses.length == 0) {
- return;
- }
- boolean accepted = false;
- for (Class clz : acceptClasses) {
- if (clz.isAssignableFrom(value.getClass())) {
- accepted = true;
- break;
- }
- }
- if (!accepted) {
- results.put(name, "value is not an expected type");
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java
deleted file mode 100644
index 648e1e4..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java
+++ /dev/null
@@ -1,74 +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.HashMap;
-import java.util.Map;
-
-/**
- * This class represents errors found when validating trigger configuration.
- */
-public class TriggerValidationException extends Exception {
- private final Map<String, String> details = new HashMap<>();
- private final String name;
-
- /**
- * Create an exception.
- * @param name name of the trigger / action / listener that caused the exception
- * @param details details of invalid configuration - key is a property name,
- * value is an error message.
- */
- public TriggerValidationException(String name, Map<String, String> details) {
- super();
- this.name = name;
- if (details != null) {
- this.details.putAll(details);
- }
- }
-
- /**
- * Create an exception.
- * @param name name of the trigger / action / listener that caused the exception
- * @param keyValues zero or even number of arguments representing symbolic key
- * (eg. property name) and the corresponding validation error message.
- */
- public TriggerValidationException(String name, String... keyValues) {
- super();
- this.name = name;
- if (keyValues == null || keyValues.length == 0) {
- return;
- }
- if (keyValues.length % 2 != 0) {
- throw new IllegalArgumentException("number of arguments representing key & value pairs must be even");
- }
- for (int i = 0; i < keyValues.length; i += 2) {
- details.put(keyValues[i], keyValues[i + 1]);
- }
- }
-
- public Map<String, String> getDetails() {
- return details;
- }
-
- @Override
- public String toString() {
- return "TriggerValidationException{" +
- "name=" + name +
- ", details='" + details + '\'' +
- '}';
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/package-info.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/package-info.java
deleted file mode 100644
index d3447aa..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/package-info.java
+++ /dev/null
@@ -1,21 +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 for classes related to autoscaling
- */
-package org.apache.solr.cloud.autoscaling;
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/ActionError.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/ActionError.java
deleted file mode 100644
index c1c070d..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/ActionError.java
+++ /dev/null
@@ -1,24 +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.sim;
-
-/**
- * Interface that helps simulating action errors.
- */
-public interface ActionError {
- boolean shouldFail(String... args);
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java
deleted file mode 100644
index fbe66ac..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/FakeDocIterator.java
+++ /dev/null
@@ -1,56 +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.sim;
-
-import java.util.Iterator;
-
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.SolrInputField;
-
-/**
- * Lightweight generator of fake documents
- * NOTE: this iterator only ever returns the same document N times, which works ok
- * for our "bulk index update" simulation. Obviously don't use this for real indexing.
- */
-public class FakeDocIterator implements Iterator<SolrInputDocument> {
- final SolrInputDocument doc = new SolrInputDocument();
- final SolrInputField idField = new SolrInputField("id");
-
- final long start, count;
-
- long current, max;
-
- FakeDocIterator(long start, long count) {
- this.start = start;
- this.count = count;
- current = start;
- max = start + count;
- doc.put("id", idField);
- idField.setValue("foo");
- }
-
- @Override
- public boolean hasNext() {
- return current < max;
- }
-
- @Override
- public SolrInputDocument next() {
- current++;
- return doc;
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
deleted file mode 100644
index 109c516..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
+++ /dev/null
@@ -1,601 +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.sim;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.function.Predicate;
-
-import com.codahale.metrics.Timer;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.solr.client.solrj.cloud.DistributedQueue;
-import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
-import org.apache.solr.client.solrj.cloud.DistribStateManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
-import org.apache.solr.cloud.OverseerTaskQueue;
-import org.apache.solr.cloud.Stats;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.util.Pair;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A distributed queue that uses {@link DistribStateManager} as the underlying distributed store.
- * Implementation based on {@link org.apache.solr.cloud.ZkDistributedQueue}
- */
-public class GenericDistributedQueue implements DistributedQueue {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- static final String PREFIX = "qn-";
-
- /**
- * Theory of operation:
- * <p>
- * Under ordinary circumstances we neither watch nor poll for children in ZK.
- * Instead we keep an in-memory list of known child names. When the in-memory
- * list is exhausted, we then fetch from ZK.
- * <p>
- * We only bother setting a child watcher when the queue has no children in ZK.
- */
- private static final Object _IMPLEMENTATION_NOTES = null;
-
- final String dir;
-
- final DistribStateManager stateManager;
-
- final Stats stats;
-
- /**
- * A lock that guards all of the mutable state that follows.
- */
- private final ReentrantLock updateLock = new ReentrantLock();
-
- /**
- * Contains the last set of children fetched from ZK. Elements are removed from the head of
- * this in-memory set as they are consumed from the queue. Due to the distributed nature
- * of the queue, elements may appear in this set whose underlying nodes have been consumed in ZK.
- * Therefore, methods like {@link #peek()} have to double-check actual node existence, and methods
- * like {@link #poll()} must resolve any races by attempting to delete the underlying node.
- */
- private TreeSet<String> knownChildren = new TreeSet<>();
-
- /**
- * Used to wait on ZK changes to the child list; you must hold {@link #updateLock} before waiting on this condition.
- */
- private final Condition changed = updateLock.newCondition();
-
- private boolean isDirty = true;
-
- private int watcherCount = 0;
-
- private final int maxQueueSize;
-
- /**
- * If {@link #maxQueueSize} is set, the number of items we can queue without rechecking the server.
- */
- private final AtomicInteger offerPermits = new AtomicInteger(0);
-
- public GenericDistributedQueue(DistribStateManager stateManager, String dir) {
- this(stateManager, dir, new Stats());
- }
-
- public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats) {
- this(stateManager, dir, stats, 0);
- }
-
- public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats, int maxQueueSize) {
- this.dir = dir;
-
- try {
- if (!stateManager.hasData(dir)) {
- try {
- stateManager.makePath(dir);
- } catch (AlreadyExistsException e) {
- // ignore
- }
- }
- } catch (IOException | KeeperException e) {
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- }
-
- this.stateManager = stateManager;
- this.stats = stats;
- this.maxQueueSize = maxQueueSize;
- }
-
- /**
- * Returns the data at the first element of the queue, or null if the queue is
- * empty.
- *
- * @return data at the first element of the queue, or null.
- */
- @Override
- public byte[] peek() throws Exception {
- Timer.Context time = stats.time(dir + "_peek");
- try {
- return firstElement();
- } finally {
- time.stop();
- }
- }
-
- /**
- * Returns the data at the first element of the queue, or null if the queue is
- * empty and block is false.
- *
- * @param block if true, blocks until an element enters the queue
- * @return data at the first element of the queue, or null.
- */
- @Override
- public byte[] peek(boolean block) throws Exception {
- return block ? peek(Long.MAX_VALUE) : peek();
- }
-
- /**
- * Returns the data at the first element of the queue, or null if the queue is
- * empty after wait ms.
- *
- * @param wait max wait time in ms.
- * @return data at the first element of the queue, or null.
- */
- @Override
- public byte[] peek(long wait) throws Exception {
- Preconditions.checkArgument(wait > 0);
- Timer.Context time;
- if (wait == Long.MAX_VALUE) {
- time = stats.time(dir + "_peek_wait_forever");
- } else {
- time = stats.time(dir + "_peek_wait" + wait);
- }
- updateLock.lockInterruptibly();
- try {
- long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
- while (waitNanos > 0) {
- byte[] result = firstElement();
- if (result != null) {
- return result;
- }
- waitNanos = changed.awaitNanos(waitNanos);
- }
- return null;
- } finally {
- updateLock.unlock();
- time.stop();
- }
- }
-
- /**
- * Attempts to remove the head of the queue and return it. Returns null if the
- * queue is empty.
- *
- * @return Head of the queue or null.
- */
- @Override
- public byte[] poll() throws Exception {
- Timer.Context time = stats.time(dir + "_poll");
- try {
- return removeFirst();
- } finally {
- time.stop();
- }
- }
-
- /**
- * Attempts to remove the head of the queue and return it.
- *
- * @return The former head of the queue
- */
- @Override
- public byte[] remove() throws Exception {
- Timer.Context time = stats.time(dir + "_remove");
- try {
- byte[] result = removeFirst();
- if (result == null) {
- throw new NoSuchElementException();
- }
- return result;
- } finally {
- time.stop();
- }
- }
-
- public void remove(Collection<String> paths) throws Exception {
- if (paths.isEmpty()) return;
- List<Op> ops = new ArrayList<>();
- for (String path : paths) {
- ops.add(Op.delete(dir + "/" + path, -1));
- }
- for (int from = 0; from < ops.size(); from += 1000) {
- int to = Math.min(from + 1000, ops.size());
- if (from < to) {
- try {
- stateManager.multi(ops.subList(from, to));
- } catch (NoSuchElementException e) {
- // don't know which nodes are not exist, so try to delete one by one node
- for (int j = from; j < to; j++) {
- try {
- stateManager.removeData(ops.get(j).getPath(), -1);
- } catch (NoSuchElementException e2) {
- if (log.isDebugEnabled()) {
- log.debug("Can not remove node which is not exist : {}", ops.get(j).getPath());
- }
- }
- }
- }
- }
- }
-
- int cacheSizeBefore = knownChildren.size();
- knownChildren.removeAll(paths);
- if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
- stats.setQueueLength(knownChildren.size());
- } else {
- // There are elements get deleted but not present in the cache,
- // the cache seems not valid anymore
- knownChildren.clear();
- isDirty = true;
- }
- }
-
- /**
- * Removes the head of the queue and returns it, blocks until it succeeds.
- *
- * @return The former head of the queue
- */
- @Override
- public byte[] take() throws Exception {
- // Same as for element. Should refactor this.
- Timer.Context timer = stats.time(dir + "_take");
- updateLock.lockInterruptibly();
- try {
- while (true) {
- byte[] result = removeFirst();
- if (result != null) {
- return result;
- }
- changed.await();
- }
- } finally {
- updateLock.unlock();
- timer.stop();
- }
- }
-
- /**
- * Inserts data into queue. If there are no other queue consumers, the offered element
- * will be immediately visible when this method returns.
- */
- @Override
- public void offer(byte[] data) throws Exception {
- Timer.Context time = stats.time(dir + "_offer");
- try {
- while (true) {
- try {
- if (maxQueueSize > 0) {
- if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) {
- // If a max queue size is set, check it before creating a new queue item.
- if (!stateManager.hasData(dir)) {
- // jump to the code below, which tries to create dir if it doesn't exist
- throw new NoSuchElementException();
- }
- List<String> children = stateManager.listData(dir);
- int remainingCapacity = maxQueueSize - children.size();
- if (remainingCapacity <= 0) {
- throw new IllegalStateException("queue is full");
- }
-
- // Allow this client to push up to 1% of the remaining queue capacity without rechecking.
- offerPermits.set(remainingCapacity / 100);
- }
- }
-
- // Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
- // This will get set again when the watcher actually fires, but that's ok.
- stateManager.createData(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL);
- isDirty = true;
- return;
- } catch (NoSuchElementException e) {
- try {
- stateManager.createData(dir, new byte[0], CreateMode.PERSISTENT);
- } catch (NoSuchElementException ne) {
- // someone created it
- }
- }
- }
- } finally {
- time.stop();
- }
- }
-
- public Stats getZkStats() {
- return stats;
- }
-
- @Override
- public Map<String, Object> getStats() {
- if (stats == null) {
- return Collections.emptyMap();
- }
- Map<String, Object> res = new HashMap<>();
- res.put("queueLength", stats.getQueueLength());
- final Map<String, Object> statsMap = new HashMap<>();
- res.put("stats", statsMap);
- stats.getStats().forEach((op, stat) -> {
- final Map<String, Object> statMap = new HashMap<>();
- statMap.put("success", stat.success.get());
- statMap.put("errors", stat.errors.get());
- final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
- statMap.put("failureDetails", failed);
- stat.failureDetails.forEach(failedOp -> {
- Map<String, Object> fo = new HashMap<>();
- fo.put("req", failedOp.req);
- fo.put("resp", failedOp.resp);
- });
- statsMap.put(op, statMap);
- });
- return res;
- }
-
- /**
- * Returns the name if the first known child node, or {@code null} if the queue is empty.
- * This is the only place {@link #knownChildren} is ever updated!
- * The caller must double check that the actual node still exists, since the in-memory
- * list is inherently stale.
- */
- private String firstChild(boolean remove, boolean refetchIfDirty) throws Exception {
- updateLock.lockInterruptibly();
- try {
- // We always return from cache first, the cache will be cleared if the node is not exist
- if (!knownChildren.isEmpty() && !(isDirty && refetchIfDirty)) {
- return remove ? knownChildren.pollFirst() : knownChildren.first();
- }
-
- if (!isDirty && knownChildren.isEmpty()) {
- return null;
- }
-
- // Dirty, try to fetch an updated list of children from ZK.
- // Only set a new watcher if there isn't already a watcher.
- ChildWatcher newWatcher = (watcherCount == 0) ? new ChildWatcher() : null;
- knownChildren = fetchZkChildren(newWatcher);
- if (newWatcher != null) {
- watcherCount++; // watcher was successfully set
- }
- isDirty = false;
- if (knownChildren.isEmpty()) {
- return null;
- }
- changed.signalAll();
- return remove ? knownChildren.pollFirst() : knownChildren.first();
- } finally {
- updateLock.unlock();
- }
- }
-
- /**
- * Return the current set of children from ZK; does not change internal state.
- */
- TreeSet<String> fetchZkChildren(Watcher watcher) throws Exception {
- while (true) {
- try {
- TreeSet<String> orderedChildren = new TreeSet<>();
-
- List<String> childNames = stateManager.listData(dir, watcher);
- stats.setQueueLength(childNames.size());
- for (String childName : childNames) {
- // Check format
- if (!childName.regionMatches(0, PREFIX, 0, PREFIX.length())) {
- log.debug("Found child node with improper name: {}", childName);
- continue;
- }
- orderedChildren.add(childName);
- }
- return orderedChildren;
- } catch (NoSuchElementException e) {
- try {
- stateManager.makePath(dir);
- } catch (AlreadyExistsException e2) {
- // ignore
- }
- // go back to the loop and try again
- }
- }
- }
-
- /**
- * Return the currently-known set of elements, using child names from memory. If no children are found, or no
- * children pass {@code acceptFilter}, waits up to {@code waitMillis} for at least one child to become available.
- * <p>
- * Package-private to support {@link OverseerTaskQueue} specifically.</p>
- */
- @Override
- public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
- List<String> foundChildren = new ArrayList<>();
- long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
- boolean first = true;
- while (true) {
- // Trigger a refresh, but only force it if this is not the first iteration.
- firstChild(false, !first);
-
- updateLock.lockInterruptibly();
- try {
- for (String child : knownChildren) {
- if (acceptFilter.test(child)) {
- foundChildren.add(child);
- }
- }
- if (!foundChildren.isEmpty()) {
- break;
- }
- if (waitNanos <= 0) {
- break;
- }
-
- // If this is our first time through, force a refresh before waiting.
- if (first) {
- first = false;
- continue;
- }
-
- waitNanos = changed.awaitNanos(waitNanos);
- } finally {
- updateLock.unlock();
- }
-
- if (!foundChildren.isEmpty()) {
- break;
- }
- }
-
- // Technically we could restart the method if we fail to actually obtain any valid children
- // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
- // much more sophisticated waitNanos tracking.
- List<Pair<String, byte[]>> result = new ArrayList<>();
- for (String child : foundChildren) {
- if (result.size() >= max) {
- break;
- }
- try {
- VersionedData data = stateManager.getData(dir + "/" + child);
- result.add(new Pair<>(child, data.getData()));
- } catch (NoSuchElementException e) {
- // Another client deleted the node first, remove the in-memory and continue.
- updateLock.lockInterruptibly();
- try {
- knownChildren.remove(child);
- } finally {
- updateLock.unlock();
- }
- }
- }
- return result;
- }
-
- /**
- * Return the head of the queue without modifying the queue.
- *
- * @return the data at the head of the queue.
- */
- private byte[] firstElement() throws Exception {
- while (true) {
- String firstChild = firstChild(false, false);
- if (firstChild == null) {
- return null;
- }
- try {
- VersionedData data = stateManager.getData(dir + "/" + firstChild);
- return data != null ? data.getData() : null;
- } catch (NoSuchElementException e) {
- // Another client deleted the node first, remove the in-memory and retry.
- updateLock.lockInterruptibly();
- try {
- // Efficient only for single-consumer
- knownChildren.clear();
- isDirty = true;
- } finally {
- updateLock.unlock();
- }
- }
- }
- }
-
- private byte[] removeFirst() throws Exception {
- while (true) {
- String firstChild = firstChild(true, false);
- if (firstChild == null) {
- return null;
- }
- try {
- String path = dir + "/" + firstChild;
- VersionedData result = stateManager.getData(path);
- stateManager.removeData(path, -1);
- stats.setQueueLength(knownChildren.size());
- return result.getData();
- } catch (NoSuchElementException e) {
- // Another client deleted the node first, remove the in-memory and retry.
- updateLock.lockInterruptibly();
- try {
- // Efficient only for single-consumer
- knownChildren.clear();
- isDirty = true;
- } finally {
- updateLock.unlock();
- }
- }
- }
- }
-
- @VisibleForTesting int watcherCount() throws InterruptedException {
- updateLock.lockInterruptibly();
- try {
- return watcherCount;
- } finally {
- updateLock.unlock();
- }
- }
-
- @VisibleForTesting boolean isDirty() throws InterruptedException {
- updateLock.lockInterruptibly();
- try {
- return isDirty;
- } finally {
- updateLock.unlock();
- }
- }
-
- @VisibleForTesting class ChildWatcher implements Watcher {
-
- @Override
- public void process(WatchedEvent event) {
- // session events are not change events, and do not remove the watcher; except for Expired
- if (Event.EventType.None.equals(event.getType()) && !Event.KeeperState.Expired.equals(event.getState())) {
- return;
- }
- updateLock.lock();
- try {
- isDirty = true;
- watcherCount--;
- // optimistically signal any waiters that the queue may not be empty now, so they can wake up and retry
- changed.signalAll();
- } finally {
- updateLock.unlock();
- }
- }
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
deleted file mode 100644
index d4d7e2f..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
+++ /dev/null
@@ -1,45 +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.sim;
-
-import java.io.IOException;
-
-import org.apache.solr.client.solrj.cloud.DistributedQueue;
-import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
-import org.apache.solr.client.solrj.cloud.DistribStateManager;
-
-/**
- * Factory for {@link GenericDistributedQueue}.
- */
-public class GenericDistributedQueueFactory implements DistributedQueueFactory {
-
- private final DistribStateManager stateManager;
-
- public GenericDistributedQueueFactory(DistribStateManager stateManager) {
- this.stateManager = stateManager;
- }
-
- @Override
- public DistributedQueue makeQueue(String path) throws IOException {
- return new GenericDistributedQueue(stateManager, path);
- }
-
- @Override
- public void removeQueue(String path) throws IOException {
-
- }
-}
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
deleted file mode 100644
index 5f12004..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
+++ /dev/null
@@ -1,113 +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.sim;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.solr.common.cloud.LiveNodesListener;
-
-/**
- * This class represents a set of live nodes and allows adding listeners to track their state.
- */
-public class LiveNodesSet implements Iterable<String> {
-
- private final Set<String> set = ConcurrentHashMap.newKeySet();
- private final Set<LiveNodesListener> listeners = ConcurrentHashMap.newKeySet();
-
- public Set<String> get() {
- return Collections.unmodifiableSet(set);
- }
-
- public int size() {
- return set.size();
- }
-
- public void registerLiveNodesListener(LiveNodesListener listener) {
- listeners.add(listener);
- }
-
- public void removeLiveNodesListener(LiveNodesListener listener) {
- listeners.remove(listener);
- }
... 74551 lines suppressed ...