mirror of https://github.com/apache/lucene.git
SOLR-11748: Remove Autoscaling action throttle
This commit is contained in:
parent
bdfbe433a3
commit
caa731a333
|
@ -49,6 +49,14 @@ Apache UIMA 2.3.1
|
|||
Apache ZooKeeper 3.4.10
|
||||
Jetty 9.3.20.v20170531
|
||||
|
||||
Upgrade Notes
|
||||
----------------------
|
||||
|
||||
* SOLR-11748: The throttling mechanism used to limit the rate of autoscaling events processed
|
||||
has been removed. This deprecates the 'actionThrottlePeriodSeconds' setting in the set-properties
|
||||
Autoscaling API which is now a no-op. Use the 'triggerCooldownPeriodSeconds' instead to pause event
|
||||
processing.
|
||||
|
||||
New Features
|
||||
----------------------
|
||||
* SOLR-11285: Simulation framework for autoscaling. (ab)
|
||||
|
@ -95,6 +103,8 @@ Other Changes
|
|||
|
||||
* SOLR-7733: remove "optimize" from the UI. (Erick Erickson)
|
||||
|
||||
* SOLR-11748: Remove Autoscaling action throttle. (shalin)
|
||||
|
||||
================== 7.2.0 ==================
|
||||
|
||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||
|
|
|
@ -39,11 +39,9 @@ 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.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.lang3.exception.ExceptionUtils;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
|
@ -53,7 +51,6 @@ 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.ActionThrottle;
|
||||
import org.apache.solr.cloud.Stats;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
|
@ -116,8 +113,6 @@ public class ScheduledTriggers implements Closeable {
|
|||
|
||||
private final AtomicLong triggerDelay = new AtomicLong(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
|
||||
|
||||
private final AtomicReference<ActionThrottle> actionThrottle;
|
||||
|
||||
private final SolrCloudManager cloudManager;
|
||||
|
||||
private final DistribStateManager stateManager;
|
||||
|
@ -136,7 +131,6 @@ public class ScheduledTriggers implements Closeable {
|
|||
scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
|
||||
scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
|
||||
actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor"));
|
||||
actionThrottle = new AtomicReference<>(new ActionThrottle("action", TimeUnit.SECONDS.toMillis(DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS), cloudManager.getTimeSource()));
|
||||
this.cloudManager = cloudManager;
|
||||
this.stateManager = cloudManager.getDistribStateManager();
|
||||
this.loader = loader;
|
||||
|
@ -183,37 +177,17 @@ public class ScheduledTriggers implements Closeable {
|
|||
case TRIGGER_CORE_POOL_SIZE:
|
||||
this.scheduledThreadPoolExecutor.setCorePoolSize(((Number) newProps.get(key)).intValue());
|
||||
break;
|
||||
case ACTION_THROTTLE_PERIOD_SECONDS:
|
||||
long minMsBetweenActions = TimeUnit.SECONDS.toMillis(((Number) newProps.get(key)).longValue());
|
||||
ActionThrottle oldThrottle = this.actionThrottle.get();
|
||||
ActionThrottle newThrottle = null;
|
||||
if (oldThrottle.getLastActionStartedAt() != null) {
|
||||
newThrottle = new ActionThrottle("action",
|
||||
minMsBetweenActions,
|
||||
oldThrottle.getLastActionStartedAt(),
|
||||
cloudManager.getTimeSource());
|
||||
} else {
|
||||
newThrottle = new ActionThrottle("action", minMsBetweenActions, cloudManager.getTimeSource());
|
||||
}
|
||||
this.actionThrottle.set(newThrottle);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
this.autoScalingConfig = autoScalingConfig;
|
||||
|
||||
// reset cooldown and actionThrottle
|
||||
// reset cooldown
|
||||
cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
|
||||
actionThrottle.get().reset();
|
||||
|
||||
listeners.setAutoScalingConfig(autoScalingConfig);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void resetActionThrottle() {
|
||||
actionThrottle.get().reset();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
|
@ -308,11 +282,6 @@ public class ScheduledTriggers implements Closeable {
|
|||
long eventProcessingStart = cloudManager.getTimeSource().getTime();
|
||||
log.debug("-- processing actions for " + event);
|
||||
try {
|
||||
// let the action executor thread wait instead of the trigger thread so we use the throttle here
|
||||
ActionThrottle actionThrottle = this.actionThrottle.get();
|
||||
actionThrottle.minimumWaitBetweenActions();
|
||||
actionThrottle.markAttemptingAction();
|
||||
|
||||
// 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);
|
||||
|
|
|
@ -1347,24 +1347,6 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
|
|||
for (int i = 0; i < 8; i++) {
|
||||
scheduledTriggers.remove(triggerList.get(i).getName());
|
||||
}
|
||||
|
||||
config = config.withProperties(Collections.singletonMap(AutoScalingParams.ACTION_THROTTLE_PERIOD_SECONDS, 6));
|
||||
scheduledTriggers.setAutoScalingConfig(config);
|
||||
lastActionExecutedAt.set(0);
|
||||
throttlingDelayMs.set(TimeUnit.SECONDS.toMillis(6));
|
||||
triggerFiredLatch = new CountDownLatch(2);
|
||||
Map<String, Object> props = map("waitFor", 0L, "actions", Collections.singletonList(map("name","throttler", "class", ThrottlingTesterAction.class.getName())));
|
||||
scheduledTriggers.add(new NodeAddedTrigger("y1", props, resourceLoader, solrCloudManager));
|
||||
scheduledTriggers.add(new NodeAddedTrigger("y2", props, resourceLoader, solrCloudManager));
|
||||
scheduledTriggers.resetActionThrottle();
|
||||
JettySolrRunner newNode = cluster.startJettySolrRunner();
|
||||
assertTrue(getTriggerFiredLatch().await(20, TimeUnit.SECONDS));
|
||||
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
|
||||
if (cluster.getJettySolrRunner(i) == newNode) {
|
||||
cluster.stopJettySolrRunner(i);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -594,9 +594,6 @@ Solr pauses all other triggers for this cool down period after a trigger fires s
|
|||
`triggerCorePoolSize`::
|
||||
The core pool size of the `ScheduledThreadPoolExecutor` used to schedule triggers. The default is `4` threads.
|
||||
|
||||
`actionThrottlePeriodSeconds`::
|
||||
This is the minimum throttling delay between executing actions for triggers. It is guaranteed that actions for two trigger events are executed after this delay period. The default is `5` seconds.
|
||||
|
||||
The command allows setting arbitrary properties in addition to the above properties. Such arbitrary properties can be useful in custom `TriggerAction` instances.
|
||||
|
||||
.Change default `triggerScheduleDelaySeconds`
|
||||
|
|
|
@ -68,5 +68,7 @@ public interface AutoScalingParams {
|
|||
String TRIGGER_SCHEDULE_DELAY_SECONDS = "triggerScheduleDelaySeconds";
|
||||
String TRIGGER_COOLDOWN_PERIOD_SECONDS = "triggerCooldownPeriodSeconds";
|
||||
String TRIGGER_CORE_POOL_SIZE = "triggerCorePoolSize";
|
||||
|
||||
@Deprecated
|
||||
String ACTION_THROTTLE_PERIOD_SECONDS = "actionThrottlePeriodSeconds";
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue