improve concurrency patterns for activity execution

This commit is contained in:
Jonathan Shook 2022-12-20 20:04:17 -06:00
parent bf5a31b342
commit b9365bff72
14 changed files with 624 additions and 649 deletions

View File

@ -17,7 +17,7 @@
package io.nosqlbench.engine.api.activityapi.core;
import io.nosqlbench.engine.api.activityapi.input.Input;
import io.nosqlbench.engine.api.activityimpl.SlotStateTracker;
import io.nosqlbench.engine.api.activityimpl.MotorState;
/**
* The core threading harness within an activity.
@ -54,6 +54,7 @@ public interface Motor<T> extends Runnable, Stoppable {
* Get a description of the current slot run status.
* @return - a value from the {@link RunState} enum
*/
SlotStateTracker getSlotStateTracker();
MotorState getState();
void removeState();
}

View File

@ -29,12 +29,11 @@ import io.nosqlbench.engine.api.activityapi.input.Input;
import io.nosqlbench.engine.api.activityapi.output.Output;
import io.nosqlbench.engine.api.activityapi.ratelimits.RateLimiter;
import io.nosqlbench.api.engine.activityimpl.ActivityDef;
import io.nosqlbench.engine.api.activityimpl.SlotStateTracker;
import io.nosqlbench.engine.api.activityimpl.MotorState;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.LogManager;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import static io.nosqlbench.engine.api.activityapi.core.RunState.*;
@ -70,8 +69,8 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
private final Activity activity;
private Output output;
private final SlotStateTracker slotStateTracker;
private final AtomicReference<RunState> slotState;
private final MotorState motorState;
// private final AtomicReference<RunState> slotState;
private int stride = 1;
private OpTracker<D> opTracker;
@ -86,14 +85,13 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
* @param input A LongSupplier which provides the cycle number inputs.
*/
public CoreMotor(
Activity activity,
long slotId,
Input input) {
Activity activity,
long slotId,
Input input) {
this.activity = activity;
this.slotId = slotId;
setInput(input);
slotStateTracker = new SlotStateTracker(slotId);
slotState = slotStateTracker.getAtomicSlotState();
motorState = new MotorState(slotId, activity.getRunStateTally());
onActivityDefUpdate(activity.getActivityDef());
}
@ -107,10 +105,10 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
* @param action An LongConsumer which is applied to the input for each cycle.
*/
public CoreMotor(
Activity activity,
long slotId,
Input input,
Action action
Activity activity,
long slotId,
Input input,
Action action
) {
this(activity, slotId, input);
setAction(action);
@ -126,11 +124,11 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
* @param output An optional opTracker.
*/
public CoreMotor(
Activity activity,
long slotId,
Input input,
Action action,
Output output
Activity activity,
long slotId,
Input input,
Action action,
Output output
) {
this(activity, slotId, input);
setAction(action);
@ -178,12 +176,18 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
}
@Override
public SlotStateTracker getSlotStateTracker() {
return slotStateTracker;
public MotorState getState() {
return motorState;
}
@Override
public void removeState() {
motorState.removeState();
}
@Override
public void run() {
motorState.enterState(Starting);
try {
inputTimer = activity.getInstrumentation().getOrCreateInputTimer();
@ -195,12 +199,10 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
cycleRateLimiter = activity.getCycleLimiter();
if (slotState.get() == Finished) {
if (motorState.get() == Finished) {
logger.warn("Input was already exhausted for slot " + slotId + ", remaining in finished state.");
}
slotStateTracker.enterState(Running);
long cyclenum;
action.init();
@ -235,7 +237,8 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
strideconsumer = (StrideOutputConsumer<D>) async;
}
while (slotState.get() == Running) {
motorState.enterState(Running);
while (motorState.get() == Running) {
CycleSegment cycleSegment = null;
@ -245,7 +248,7 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
if (cycleSegment == null) {
logger.trace("input exhausted (input " + input + ") via null segment, stopping motor thread " + slotId);
slotStateTracker.enterState(Finished);
motorState.enterState(Finished);
continue;
}
@ -256,27 +259,27 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
StrideTracker<D> strideTracker = new StrideTracker<>(
strideServiceTimer,
stridesResponseTimer,
strideDelay,
cycleSegment.peekNextCycle(),
stride,
output,
strideconsumer);
stridesResponseTimer,
strideDelay,
cycleSegment.peekNextCycle(),
stride,
output,
strideconsumer);
strideTracker.start();
long strideStart = System.nanoTime();
while (!cycleSegment.isExhausted() && slotState.get() == Running) {
while (!cycleSegment.isExhausted() && motorState.get() == Running) {
cyclenum = cycleSegment.nextCycle();
if (cyclenum < 0) {
if (cycleSegment.isExhausted()) {
logger.trace("input exhausted (input " + input + ") via negative read, stopping motor thread " + slotId);
slotStateTracker.enterState(Finished);
motorState.enterState(Finished);
continue;
}
}
if (slotState.get() != Running) {
if (motorState.get() != Running) {
logger.trace("motor stopped in cycle " + cyclenum + ", stopping motor thread " + slotId);
continue;
}
@ -287,7 +290,7 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
}
try {
TrackedOp<D> op = opTracker.newOp(cyclenum,strideTracker);
TrackedOp<D> op = opTracker.newOp(cyclenum, strideTracker);
op.setWaitTime(cycleDelay);
synchronized (opTracker) {
@ -312,7 +315,7 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
}
if (slotState.get() == Finished) {
if (motorState.get() == Finished) {
boolean finished = opTracker.awaitCompletion(60000);
if (finished) {
logger.debug("slot " + this.slotId + " completed successfully");
@ -321,12 +324,12 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
}
}
if (slotState.get() == Stopping) {
slotStateTracker.enterState(Stopped);
if (motorState.get() == Stopping) {
motorState.enterState(Stopped);
}
} else if (action instanceof SyncAction) {
} else if (action instanceof SyncAction sync) {
cycleServiceTimer = activity.getInstrumentation().getOrCreateCyclesServiceTimer();
strideServiceTimer = activity.getInstrumentation().getOrCreateStridesServiceTimer();
@ -335,9 +338,8 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
throw new RuntimeException("The async parameter was given for this activity, but it does not seem to know how to do async.");
}
SyncAction sync = (SyncAction) action;
while (slotState.get() == Running) {
motorState.enterState(Running);
while (motorState.get() == Running) {
CycleSegment cycleSegment = null;
CycleResultSegmentBuffer segBuffer = new CycleResultSegmentBuffer(stride);
@ -348,7 +350,7 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
if (cycleSegment == null) {
logger.trace("input exhausted (input " + input + ") via null segment, stopping motor thread " + slotId);
slotStateTracker.enterState(Finished);
motorState.enterState(Finished);
continue;
}
@ -366,12 +368,12 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
if (cyclenum < 0) {
if (cycleSegment.isExhausted()) {
logger.trace("input exhausted (input " + input + ") via negative read, stopping motor thread " + slotId);
slotStateTracker.enterState(Finished);
motorState.enterState(Finished);
continue;
}
}
if (slotState.get() != Running) {
if (motorState.get() != Running) {
logger.trace("motor stopped after input (input " + cyclenum + "), stopping motor thread " + slotId);
continue;
}
@ -391,6 +393,9 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
result = sync.runCycle(cyclenum);
long phaseEnd = System.nanoTime();
} catch (Exception e) {
motorState.enterState(Errored);
throw e;
} finally {
long cycleEnd = System.nanoTime();
cycleServiceTimer.update((cycleEnd - cycleStart) + cycleDelay, TimeUnit.NANOSECONDS);
@ -414,25 +419,29 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
}
}
if (slotState.get() == Stopping) {
slotStateTracker.enterState(Stopped);
}
} else {
throw new RuntimeException("Valid Action implementations must implement either the SyncAction or the AsyncAction sub-interface");
}
if (motorState.get() == Stopping) {
motorState.enterState(Stopped);
logger.trace(() -> Thread.currentThread().getName() + " shutting down as " + motorState.get());
} else if (motorState.get() == Finished) {
logger.trace(() -> Thread.currentThread().getName() + " shutting down as " + motorState.get());
} else {
logger.warn("Unexpected motor state for CoreMotor shutdown: " + motorState.get());
}
} catch (Throwable t) {
logger.error("Error in core motor loop:" + t, t);
motorState.enterState(Errored);
throw t;
}
}
@Override
public String toString() {
return "slot:" + this.slotId + "; state:" + slotState.get();
return "slot:" + this.slotId + "; state:" + motorState.get();
}
@Override
@ -452,17 +461,17 @@ public class CoreMotor<D> implements ActivityDefObserver, Motor<D>, Stoppable {
@Override
public synchronized void requestStop() {
if (slotState.get() == Running) {
if (motorState.get() == Running) {
if (input instanceof Stoppable) {
((Stoppable) input).requestStop();
}
if (action instanceof Stoppable) {
((Stoppable) action).requestStop();
}
slotStateTracker.enterState(RunState.Stopping);
motorState.enterState(RunState.Stopping);
} else {
if (slotState.get() != Stopped && slotState.get() != Stopping) {
logger.warn("attempted to stop motor " + this.getSlotId() + ": from non Running state:" + slotState.get());
if (motorState.get() != Stopped && motorState.get() != Stopping) {
logger.warn("attempted to stop motor " + this.getSlotId() + ": from non Running state:" + motorState.get());
}
}
}

View File

@ -1,50 +0,0 @@
/*
* Copyright (c) 2022 nosqlbench
*
* Licensed 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 io.nosqlbench.engine.core.lifecycle;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class ActivityFinisher extends Thread {
private final static Logger logger = LogManager.getLogger(ActivityFinisher.class);
private final ActivityThreadsManager executor;
private final int timeout;
private boolean result;
public ActivityFinisher(ActivityThreadsManager executor, int timeout) {
super(executor.getActivityDef().getAlias() + "_finisher");
this.executor = executor;
this.timeout = timeout;
}
@Override
public void run() {
logger.debug(this + " awaiting async completion of " + executor.getActivity().getAlias() + " on " + executor + " for timeout " + timeout);
result = executor.awaitCompletion(timeout);
logger.debug(this + " awaited async completion of " + executor.getActivity().getAlias());
}
public boolean getResult() {
return result;
}
@Override
public String toString() {
return this.getClass().getSimpleName()+"/" + executor.getActivity().getAlias();
}
}

View File

@ -28,7 +28,7 @@ import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.TimeUnit;
public class ExecMetricsResult extends ExecResult {
public class ExecutionMetricsResult extends ExecutionResult {
public static final Set<MetricAttribute> INTERVAL_ONLY_METRICS = Set.of(
MetricAttribute.MIN,
@ -48,8 +48,8 @@ public class ExecMetricsResult extends ExecResult {
MetricAttribute.M15_RATE
);
public ExecMetricsResult(long startedAt, long endedAt, String iolog, Exception e) {
super(startedAt, endedAt, iolog, e);
public ExecutionMetricsResult(long startedAt, long endedAt, String iolog, Exception error) {
super(startedAt, endedAt, iolog, error);
}
public String getMetricsSummary() {

View File

@ -19,28 +19,26 @@ package io.nosqlbench.engine.core.lifecycle;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.Optional;
/**
* Provide a result type back to a caller, including the start and end times,
* any exception that occurred, and any content written to stdout or stderr equivalent
* IO streams. This is an <EM>execution result</EM>.
*
*/
public class ExecResult {
protected final static Logger logger = LogManager.getLogger(ExecMetricsResult.class);
public class ExecutionResult {
protected final static Logger logger = LogManager.getLogger(ExecutionMetricsResult.class);
protected final long startedAt;
protected final long endedAt;
protected final Exception exception;
protected final String iolog;
public ExecResult(long startedAt, long endedAt, String iolog, Exception e) {
public ExecutionResult(long startedAt, long endedAt, String iolog, Exception error) {
this.startedAt = startedAt;
this.endedAt = endedAt;
this.exception = e;
this.iolog = ((iolog != null) ? iolog + "\n\n" : "") + (e != null ? e.getMessage() : "");
logger.debug("populating "+(e==null? "NORMAL" : "ERROR")+" scenario result");
if (logger.isDebugEnabled()) {
this.exception = error;
this.iolog = ((iolog != null) ? iolog + "\n\n" : "") + exception;
logger.debug("populating "+(error==null ? "NORMAL" : "ERROR")+" scenario result");
if (logger.isTraceEnabled()) {
StackTraceElement[] st = Thread.currentThread().getStackTrace();
for (int i = 0; i < st.length; i++) {
logger.debug(":AT " + st[i].getFileName()+":"+st[i].getLineNumber()+":"+st[i].getMethodName());
@ -51,7 +49,7 @@ public class ExecResult {
}
public void reportElapsedMillisToLog() {
logger.info("-- SCENARIO TOOK " + getElapsedMillis() + "ms --");
logger.info(() -> String.format("-- SCENARIO TOOK %.3fS --",(getElapsedMillis()/1000.0f)));
}
public String getIOLog() {
@ -62,7 +60,7 @@ public class ExecResult {
return endedAt - startedAt;
}
public Optional<Exception> getException() {
return Optional.ofNullable(exception);
public Exception getException() {
return exception;
}
}

View File

@ -1,29 +0,0 @@
/*
* Copyright (c) 2022 nosqlbench
*
* Licensed 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 io.nosqlbench.engine.core.lifecycle;
import io.nosqlbench.engine.api.activityapi.core.Activity;
public class StartedActivityInfo {
private final Activity activity;
StartedActivityInfo(Activity activity) {
this.activity = activity;
}
}

View File

@ -0,0 +1,40 @@
/*
* Copyright (c) 2022 nosqlbench
*
* Licensed 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 io.nosqlbench.engine.core.lifecycle.activity;
import io.nosqlbench.engine.core.lifecycle.scenario.ScenarioController;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class ActivitiesExceptionHandler implements Thread.UncaughtExceptionHandler {
private static final Logger logger = LogManager.getLogger(ActivitiesExceptionHandler.class);
private final ScenarioController controller;
public ActivitiesExceptionHandler(ScenarioController controller) {
this.controller = controller;
logger.debug(() -> "Activity exception handler starting up for executor '" + this.controller + "'");
}
@Override
public void uncaughtException(Thread t, Throwable e) {
logger.error("Uncaught exception in thread '" + t.getName() + ", state[" + t.getState() + "], notifying executor '" + controller + "'");
controller.notifyException(t, e);
}
}

View File

@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.nosqlbench.engine.core.lifecycle;
package io.nosqlbench.engine.core.lifecycle.activity;
import io.nosqlbench.api.annotations.Annotation;
import io.nosqlbench.api.annotations.Layer;
@ -22,42 +22,48 @@ import io.nosqlbench.api.engine.activityimpl.ParameterMap;
import io.nosqlbench.engine.api.activityapi.core.*;
import io.nosqlbench.engine.api.activityapi.core.progress.ProgressCapable;
import io.nosqlbench.engine.api.activityapi.core.progress.ProgressMeterDisplay;
import io.nosqlbench.engine.api.activityimpl.motor.RunStateImage;
import io.nosqlbench.engine.api.activityimpl.motor.RunStateTally;
import io.nosqlbench.engine.core.annotation.Annotators;
import io.nosqlbench.engine.core.lifecycle.ExecutionResult;
import io.nosqlbench.engine.core.lifecycle.IndexedThreadFactory;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.*;
import java.util.stream.Collectors;
/**
* <p>An ActivityExecutor is a named instance of an execution harness for a single activity instance.
* <p>An ActivityExecutor is an execution harness for a single activity instance.
* It is responsible for managing threads and activity settings which may be changed while the activity is running.</p>
*
* <p>An ActivityExecutor may be represent an activity that is defined and active in the running
* scenario, but which is inactive. This can occur when an activity is paused by controlling logic, or when the threads
* are set to zero.</p>
* <p>In order to allow for dynamic thread management, which is not easily supported as an explicit feature
* of most executor services, threads are started as long-running processes and managed via state signaling.
* The {@link RunState} enum, {@link io.nosqlbench.engine.api.activityimpl.MotorState} type, and {@link RunStateTally}
* state tracking class are used together to represent valid states and transitions, contain and transition state atomically,
* and provide blocking conditions for observers, respectively.</p>
*
* <P>Some basic rules and invariants must be observed for consistent concurrent behavior.
* Any state changes for a Motor must be made through {@link Motor#getState()}.
* This allows the state tracking to work consistently for all observers.</p>
*
* <p>
* Invariants:
* </p>
* <ul>
* <li>Motors may not receive parameter updates before their owning activities are initialized.</li>
* </ul>
*/
public class ActivityThreadsManager implements ActivityController, ParameterMap.Listener, ProgressCapable, Callable<ExecResult> {
public class ActivityExecutor implements ActivityController, ParameterMap.Listener, ProgressCapable, Callable<ExecutionResult> {
private static final Logger logger = LogManager.getLogger(ActivityThreadsManager.class);
// TODO Encapsulate valid state transitions to be only modifiable within the appropriate type view.
private static final Logger logger = LogManager.getLogger(ActivityExecutor.class);
private static final Logger activitylogger = LogManager.getLogger("ACTIVITY");
private final List<Motor<?>> motors = new ArrayList<>();
private final Activity activity;
private final ActivityDef activityDef;
private final ExecutorService executorService;
private RuntimeException stoppingException;
private final RunStateTally tally;
private ExecutorService executorService;
private Exception exception;
private final static int waitTime = 10000;
private String sessionId = "";
@ -67,75 +73,32 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
// private RunState intendedState = RunState.Uninitialized;
public ActivityThreadsManager(Activity activity, String sessionId) {
public ActivityExecutor(Activity activity, String sessionId) {
this.activity = activity;
this.activityDef = activity.getActivityDef();
executorService = new ThreadPoolExecutor(
0, Integer.MAX_VALUE,
0L, TimeUnit.SECONDS,
new SynchronousQueue<>(),
new IndexedThreadFactory(activity.getAlias(), new ActivityExceptionHandler(this))
);
activity.getActivityDef().getParams().addListener(this);
activity.setActivityController(this);
this.sessionId = sessionId;
this.tally = activity.getRunStateTally();
}
// TODO: Doc how uninitialized activities do not propagate parameter map changes and how
// TODO: this is different from preventing modification to uninitialized activities
/**
* <p>True-up the number of motor instances known to the executor. Start all non-running motors.
* The protocol between the motors and the executor should be safe as long as each state change is owned by either
* the motor logic or the activity executor but not both, and strictly serialized as well. This is enforced by
* forcing start(...) to be serialized as well as using CAS on the motor states.</p>
* <p>The startActivity method may be called to true-up the number of active motors in an activity executor after
* changes to threads.</p>
*/
public synchronized void startActivity() {
logger.info("starting activity " + activity.getAlias() + " for cycles " + activity.getCycleSummary());
Annotators.recordAnnotation(Annotation.newBuilder()
.session(sessionId)
.now()
.layer(Layer.Activity)
.label("alias", getActivityDef().getAlias())
.label("driver", getActivityDef().getActivityType())
.label("workload", getActivityDef().getParams().getOptionalString("workload").orElse("none"))
.detail("params", getActivityDef().toString())
.build()
);
activitylogger.debug("START/before alias=(" + activity.getAlias() + ")");
try {
activity.setRunState(RunState.Starting);
this.startedAt = System.currentTimeMillis();
activity.initActivity();
//activity.onActivityDefUpdate(activityDef);
} catch (Exception e) {
this.stoppingException = new RuntimeException("Error initializing activity '" + activity.getAlias() + "':\n" + e.getMessage(), e);
// activitylogger.error("error initializing activity '" + activity.getAlias() + "': " + stoppingException);
throw stoppingException;
}
adjustToActivityDef(activity.getActivityDef());
activity.setRunState(RunState.Running);
activitylogger.debug("START/after alias=(" + activity.getAlias() + ")");
}
/**
* Simply stop the motors
*/
private synchronized void stopActivity() {
activitylogger.debug("STOP/before alias=(" + activity.getAlias() + ")");
activity.setRunState(RunState.Stopping);
public void stopActivity() {
logger.info("stopping activity in progress: " + this.getActivityDef().getAlias());
activity.setRunState(RunState.Stopping);
motors.forEach(Motor::requestStop);
motors.forEach(m -> awaitRequiredMotorState(m, 30000, 50, RunState.Stopped, RunState.Finished));
activity.shutdownActivity();
activity.closeAutoCloseables();
tally.awaitNoneOther(RunState.Stopped,RunState.Finished);
shutdownExecutorService(Integer.MAX_VALUE);
tally.awaitNoneOther(RunState.Stopped,RunState.Finished);
activity.setRunState(RunState.Stopped);
logger.info("stopped: " + this.getActivityDef().getAlias() + " with " + motors.size() + " slots");
activitylogger.debug("STOP/after alias=(" + activity.getAlias() + ")");
Annotators.recordAnnotation(Annotation.newBuilder()
.session(sessionId)
@ -149,7 +112,7 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
);
}
public RuntimeException forceStopActivity(int initialMillisToWait) {
public Exception forceStopActivity(int initialMillisToWait) {
activitylogger.debug("FORCE STOP/before alias=(" + activity.getAlias() + ")");
activity.setRunState(RunState.Stopped);
@ -188,10 +151,10 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
logger.debug("took " + (activityShutdownEndedAt - activityShutdownStartedAt) + " ms to shutdown activity threads");
activitylogger.debug("FORCE STOP/after alias=(" + activity.getAlias() + ")");
if (stoppingException != null) {
if (exception != null) {
activitylogger.debug("FORCE STOP/exception alias=(" + activity.getAlias() + ")");
}
return stoppingException;
return exception;
}
@ -201,60 +164,19 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
* @param initialMillisToWait milliseconds to wait after graceful shutdownActivity request, before forcing
* everything to stop
*/
private synchronized void forceStopScenarioAndThrow(int initialMillisToWait, boolean rethrow) {
RuntimeException exception = forceStopActivity(initialMillisToWait);
public synchronized void forceStopScenarioAndThrow(int initialMillisToWait, boolean rethrow) {
Exception exception = forceStopActivity(initialMillisToWait);
if (exception != null && rethrow) {
throw exception;
throw new RuntimeException(exception);
}
}
private boolean finishAndShutdownExecutor(int secondsToWait) {
activitylogger.debug("REQUEST STOP/before alias=(" + activity.getAlias() + ")");
logger.debug("Stopping executor for " + activity.getAlias() + " when work completes.");
boolean wasStopped = false;
try {
executorService.shutdown();
logger.trace(() -> "awaiting termination with timeout of " + secondsToWait + " seconds");
wasStopped = executorService.awaitTermination(secondsToWait, TimeUnit.SECONDS);
} catch (InterruptedException ie) {
logger.trace("interrupted while awaiting termination");
wasStopped = false;
logger.warn("while waiting termination of shutdown " + activity.getAlias() + ", " + ie.getMessage());
activitylogger.debug("REQUEST STOP/exception alias=(" + activity.getAlias() + ") wasstopped=" + wasStopped);
} catch (RuntimeException e) {
logger.trace("Received exception while awaiting termination: " + e.getMessage());
wasStopped = true;
stoppingException = e;
} finally {
logger.trace(() -> "finally shutting down activity " + this.getActivity().getAlias());
activity.shutdownActivity();
logger.trace("closing auto-closeables");
activity.closeAutoCloseables();
activity.setRunState(RunState.Stopped);
this.stoppedAt = System.currentTimeMillis();
}
if (stoppingException != null) {
logger.trace(() -> "an exception caused the activity to stop:" + stoppingException.getMessage());
logger.trace("Setting ERROR on activity executor: " + stoppingException.getMessage());
throw stoppingException;
}
activitylogger.debug("REQUEST STOP/after alias=(" + activity.getAlias() + ") wasstopped=" + wasStopped);
return wasStopped;
}
/**
* Listens for changes to parameter maps, maps them to the activity instance, and notifies all eligible listeners of
* changes.
*/
@Override
public synchronized void handleParameterMapUpdate(ParameterMap parameterMap) {
public void handleParameterMapUpdate(ParameterMap parameterMap) {
activity.onActivityDefUpdate(activityDef);
@ -263,7 +185,7 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
// by the RunState.
if (activity.getRunState() != RunState.Uninitialized) {
if (activity.getRunState() == RunState.Running) {
adjustToActivityDef(activity.getActivityDef());
adjustMotorCountToThreadParam(activity.getActivityDef());
}
motors.stream()
.filter(m -> (m instanceof ActivityDefObserver))
@ -286,7 +208,7 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
*/
private boolean awaitCompletion(int waitTime) {
logger.debug(() -> "awaiting completion of '" + this.getActivity().getAlias() + "'");
boolean finished = finishAndShutdownExecutor(waitTime);
boolean finished = shutdownExecutorService(waitTime);
Annotators.recordAnnotation(Annotation.newBuilder()
.session(sessionId)
@ -302,28 +224,13 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
return finished;
}
public boolean awaitFinishedOrStopped(int timeout) {
activitylogger.debug("AWAIT-FINISH/before alias=(" + activity.getAlias() + ")");
boolean awaited = awaitAllRequiredMotorState(timeout, 50, RunState.Finished, RunState.Stopped);
if (awaited) {
awaited = awaitCompletion(timeout);
}
if (stoppingException != null) {
activitylogger.debug("AWAIT-FINISH/exception alias=(" + activity.getAlias() + ")");
throw stoppingException;
}
activitylogger.debug("AWAIT-FINISH/after alias=(" + activity.getAlias() + ")");
return awaited;
}
public String toString() {
return getClass().getSimpleName() + "~" + activityDef.getAlias();
}
private String getSlotStatus() {
return motors.stream()
.map(m -> m.getSlotStateTracker().getSlotState().getCode())
.map(m -> m.getState().get().getCode())
.collect(Collectors.joining(",", "[", "]"));
}
@ -332,17 +239,19 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
*
* @param activityDef the activityDef for this activity instance
*/
private synchronized void adjustToActivityDef(ActivityDef activityDef) {
private void adjustMotorCountToThreadParam(ActivityDef activityDef) {
logger.trace(() -> ">-pre-adjust->" + getSlotStatus());
// Stop and remove extra motor slots
while (motors.size() > activityDef.getThreads()) {
Motor motor = motors.get(motors.size() - 1);
logger.trace(() -> "Stopping cycle motor thread:" + motor);
motor.requestStop();
motors.remove(motors.size() - 1);
}
reduceActiveMotorCountDownToThreadParam(activityDef);
increaseActiveMotorCountUpToThreadParam(activityDef);
alignMotorStateToIntendedActivityState();
awaitAlignmentOfMotorStateToActivityState();
logger.trace(() -> ">post-adjust->" + getSlotStatus());
}
private void increaseActiveMotorCountUpToThreadParam(ActivityDef activityDef) {
// Create motor slots
while (motors.size() < activityDef.getThreads()) {
@ -350,15 +259,27 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
logger.trace(() -> "Starting cycle motor thread:" + motor);
motors.add(motor);
}
applyIntendedStateToDivergentMotors();
awaitActivityAndMotorStateAlignment();
logger.trace(() -> ">post-adjust->" + getSlotStatus());
}
private void applyIntendedStateToDivergentMotors() {
private void reduceActiveMotorCountDownToThreadParam(ActivityDef activityDef) {
// Stop and remove extra motor slots
while (motors.size() > activityDef.getThreads()) {
Motor motor = motors.get(motors.size() - 1);
logger.trace(() -> "Stopping cycle motor thread:" + motor);
motor.requestStop();
motor.removeState();
/**
* NOTE: this leaves trailing, longer-running threads which might hold the executor open
* to potentially be cleaned up by {@link ExecutorService#shutdown()} or
* {@link ExecutorService#shutdownNow()}. At this point, the motor thread has
* been instructed to shutdown, and it is effectively thread-non-grata to the activity.
*/
motors.remove(motors.size() - 1);
}
}
private void alignMotorStateToIntendedActivityState() {
RunState intended = activity.getRunState();
logger.trace(() -> "ADJUSTING to INTENDED " + intended);
switch (intended) {
@ -367,17 +288,16 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
case Running:
case Starting:
motors.stream()
.filter(m -> m.getSlotStateTracker().getSlotState() != RunState.Running)
.filter(m -> m.getSlotStateTracker().getSlotState() != RunState.Finished)
.filter(m -> m.getSlotStateTracker().getSlotState() != RunState.Starting)
.filter(m -> m.getState().get() != RunState.Running)
.filter(m -> m.getState().get() != RunState.Finished)
.filter(m -> m.getState().get() != RunState.Starting)
.forEach(m -> {
m.getSlotStateTracker().enterState(RunState.Starting);
executorService.execute(m);
});
break;
case Stopped:
motors.stream()
.filter(m -> m.getSlotStateTracker().getSlotState() != RunState.Stopped)
.filter(m -> m.getState().get() != RunState.Stopped)
.forEach(Motor::requestStop);
break;
case Finished:
@ -389,20 +309,21 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
}
}
private void awaitActivityAndMotorStateAlignment() {
private void awaitAlignmentOfMotorStateToActivityState() {
logger.debug(()->"awaiting state alignment from " + activity.getRunState());
switch (activity.getRunState()) {
case Starting:
case Running:
motors.forEach(m -> awaitRequiredMotorState(m, waitTime, 50, RunState.Running, RunState.Finished));
tally.awaitNoneOther(RunState.Running, RunState.Finished);
break;
case Stopped:
motors.forEach(m -> awaitRequiredMotorState(m, waitTime, 50, RunState.Stopped, RunState.Finished));
tally.awaitNoneOther(RunState.Stopped, RunState.Finished);
break;
case Uninitialized:
break;
case Finished:
motors.forEach(m -> awaitRequiredMotorState(m, waitTime, 50, RunState.Finished));
tally.awaitNoneOther(RunState.Finished);
break;
case Stopping:
throw new RuntimeException("Invalid requested state in activity executor:" + activity.getRunState());
@ -413,106 +334,40 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
}
/**
* Await a thread (aka motor/slot) entering a specific SlotState
*
* @param m motor instance
* @param waitTime milliseconds to wait, total
* @param pollTime polling interval between state checks
* @param desiredRunStates any desired SlotState
* @return true, if the desired SlotState was detected
*/
private boolean awaitMotorState(Motor<?> m, int waitTime, int pollTime, RunState... desiredRunStates) {
long startedAt = System.currentTimeMillis();
while (System.currentTimeMillis() < (startedAt + waitTime)) {
for (RunState desiredRunState : desiredRunStates) {
if (desiredRunState == m.getSlotStateTracker().getSlotState()) {
return true;
}
}
try {
Thread.sleep(pollTime);
} catch (InterruptedException ignored) {
}
}
logger.trace(() -> activityDef.getAlias() + "/Motor[" + m.getSlotId() + "] is now in state " + m.getSlotStateTracker().getSlotState());
return false;
}
private boolean awaitAllRequiredMotorState(int waitTime, int pollTime, RunState... awaitingState) {
long startedAt = System.currentTimeMillis();
boolean awaited = false;
while (!awaited && (System.currentTimeMillis() < (startedAt + waitTime))) {
awaited = true;
for (Motor motor : motors) {
awaited = awaitMotorState(motor, waitTime, pollTime, awaitingState);
if (!awaited) {
logger.trace(() -> "failed awaiting motor " + motor.getSlotId() + " for state in " +
Arrays.asList(awaitingState));
break;
}
}
}
return awaited;
}
/**
* Await a required thread (aka motor/slot) entering a specific SlotState
*
* @param m motor instance
* @param waitTime milliseconds to wait, total
* @param pollTime polling interval between state checks
* @param awaitingState desired SlotState
* @throws RuntimeException if the waitTime is used up and the desired state is not reached
*/
private void awaitRequiredMotorState(Motor m, int waitTime, int pollTime, RunState... awaitingState) {
RunState startingState = m.getSlotStateTracker().getSlotState();
boolean awaitedRequiredState = awaitMotorState(m, waitTime, pollTime, awaitingState);
if (!awaitedRequiredState) {
String error = "Unable to await " + activityDef.getAlias() +
"/Motor[" + m.getSlotId() + "]: from state " + startingState + " to " + m.getSlotStateTracker().getSlotState()
+ " after waiting for " + waitTime + "ms";
RuntimeException e = new RuntimeException(error);
logger.error(error);
throw e;
}
logger.trace(() -> "motor " + m + " entered awaited state: " + Arrays.asList(awaitingState));
}
private synchronized void requestStopMotors() {
private void requestStopMotors() {
logger.info("stopping activity " + activity);
activity.setRunState(RunState.Stopped);
activity.setRunState(RunState.Stopping);
motors.forEach(Motor::requestStop);
}
public boolean isRunning() {
return motors.stream().anyMatch(m -> m.getSlotStateTracker().getSlotState() == RunState.Running);
return motors.stream().anyMatch(m -> m.getState().get() == RunState.Running);
}
public Activity getActivity() {
return activity;
}
public synchronized void notifyException(Thread t, Throwable e) {
public void notifyException(Thread t, Throwable e) {
logger.debug(() -> "Uncaught exception in activity thread forwarded to activity executor: " + e.getMessage());
this.stoppingException = new RuntimeException("Error in activity thread " + t.getName(), e);
forceStopActivity(10000);
this.exception = new RuntimeException("Error in activity thread " + t.getName(), e);
this.requestStopMotors();
}
@Override
public synchronized void stopActivityWithReasonAsync(String reason) {
logger.info("Stopping activity " + this.activityDef.getAlias() + ": " + reason);
this.stoppingException = new RuntimeException("Stopping activity " + this.activityDef.getAlias() + ": " + reason);
logger.error("stopping with reason: " + stoppingException);
this.exception = new RuntimeException("Stopping activity " + this.activityDef.getAlias() + ": " + reason);
logger.error("stopping with reason: " + exception);
requestStopMotors();
}
@Override
public synchronized void stopActivityWithErrorAsync(Throwable throwable) {
if (stoppingException == null) {
this.stoppingException = new RuntimeException(throwable);
if (exception == null) {
this.exception = new RuntimeException(throwable);
logger.error("stopping on error: " + throwable.toString(), throwable);
} else {
if (activityDef.getParams().getOptionalBoolean("fullerrors").orElse(false)) {
@ -531,9 +386,139 @@ public class ActivityThreadsManager implements ActivityController, ParameterMap.
@Override
public synchronized ExecResult call() throws Exception {
boolean stopped = awaitCompletion(Integer.MAX_VALUE);
ExecResult result = new ExecResult(startedAt, stoppedAt, "", this.stoppingException);
public ExecutionResult call() throws Exception {
try {
// instantiate and configure fixtures that need to be present
// before threads start running such as metrics instruments
activity.initActivity();
awaitMotorsAtLeastRunning();
awaitActivityCompletion();
activity.shutdownActivity();
activity.closeAutoCloseables();
} catch (Exception e) {
this.exception = e;
}
ExecutionResult result = new ExecutionResult(startedAt, stoppedAt, "", exception);
return result;
}
/**
* This waits for at least one motor to be in running, finished or stopped state.
* A motor with enough cycles to read will go into a running state. A motor which has
* a short read immediately after being started will go into a finished state. A motor
* which has been stopped for some reason, like an error or a stop command will go into
* stopped state. All of these states are sufficient to signal that successful startup
* has been completed at least.
*/
private void awaitMotorsAtLeastRunning() {
RunStateImage states = tally.awaitAny(RunState.Running, RunState.Stopped, RunState.Finished, RunState.Errored);
RunState maxState = states.getMaxState();
if (maxState==RunState.Errored) {
activity.setRunState(maxState);
throw new RuntimeException("Error in activity");
}
}
public void startActivity() {
// we need an executor service to run motor threads on
startMotorExecutorService();
startRunningActivityThreads();
awaitMotorsAtLeastRunning();
}
private boolean shutdownExecutorService(int secondsToWait) {
activitylogger.debug(() -> "Shutting down motor executor for (" + activity.getAlias() + ")");
boolean wasStopped = false;
try {
executorService.shutdown();
logger.trace(() -> "awaiting termination with timeout of " + secondsToWait + " seconds");
wasStopped = executorService.awaitTermination(secondsToWait, TimeUnit.SECONDS);
} catch (InterruptedException ie) {
logger.trace("interrupted while awaiting termination");
wasStopped = false;
logger.warn("while waiting termination of shutdown " + activity.getAlias() + ", " + ie.getMessage());
activitylogger.debug("REQUEST STOP/exception alias=(" + activity.getAlias() + ") wasstopped=" + wasStopped);
} catch (RuntimeException e) {
logger.trace("Received exception while awaiting termination: " + e.getMessage());
wasStopped = true;
exception = e;
} finally {
logger.trace(() -> "finally shutting down activity " + this.getActivity().getAlias());
this.stoppedAt = System.currentTimeMillis();
activity.setRunState(RunState.Stopped);
}
if (exception != null) {
logger.trace(() -> "an exception caused the activity to stop:" + exception.getMessage());
logger.warn("Setting ERROR on motor executor for activity '" + activity.getAlias() + "': " + exception.getMessage());
throw new RuntimeException(exception);
}
activitylogger.debug("motor executor for " + activity.getAlias() + ") wasstopped=" + wasStopped);
return wasStopped;
}
private void awaitActivityCompletion() {
RunStateImage state = tally.awaitNoneOther(RunState.Stopped, RunState.Finished, RunState.Errored);
RunState maxState = state.getMaxState();
activity.setRunState(maxState);
if (maxState==RunState.Errored) {
throw new RuntimeException("Error while waiting for activity completion:" + this.exception);
}
}
private void startMotorExecutorService() {
this.executorService = new ThreadPoolExecutor(
0, Integer.MAX_VALUE,
0L, TimeUnit.SECONDS,
new SynchronousQueue<>(),
new IndexedThreadFactory(activity.getAlias(), new ActivityExceptionHandler(this))
);
}
/**
* <p>True-up the number of motor instances known to the executor. Start all non-running motors.
* The protocol between the motors and the executor should be safe as long as each state change is owned by either
* the motor logic or the activity executor but not both, and strictly serialized as well. This is enforced by
* forcing start(...) to be serialized as well as using CAS on the motor states.</p>
* <p>The startActivity method may be called to true-up the number of active motors in an activity executor after
* changes to threads.</p>
*/
private void startRunningActivityThreads() {
logger.info("starting activity " + activity.getAlias() + " for cycles " + activity.getCycleSummary());
Annotators.recordAnnotation(Annotation.newBuilder()
.session(sessionId)
.now()
.layer(Layer.Activity)
.label("alias", getActivityDef().getAlias())
.label("driver", getActivityDef().getActivityType())
.label("workload", getActivityDef().getParams().getOptionalString("workload").orElse("none"))
.detail("params", getActivityDef().toString())
.build()
);
activitylogger.debug("START/before alias=(" + activity.getAlias() + ")");
try {
activity.setRunState(RunState.Starting);
this.startedAt = System.currentTimeMillis();
activity.onActivityDefUpdate(activityDef);
} catch (Exception e) {
this.exception = new RuntimeException("Error initializing activity '" + activity.getAlias() + "':\n" + e.getMessage(), e);
activitylogger.error(()->"error initializing activity '" + activity.getAlias() + "': " + exception);
throw new RuntimeException(exception);
}
adjustMotorCountToThreadParam(activity.getActivityDef());
tally.awaitAny(RunState.Running,RunState.Finished,RunState.Stopped);
activity.setRunState(RunState.Running);
activitylogger.debug("START/after alias=(" + activity.getAlias() + ")");
}
}

View File

@ -0,0 +1,97 @@
/*
* Copyright (c) 2022 nosqlbench
*
* Licensed 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 io.nosqlbench.engine.core.lifecycle.activity;
import io.nosqlbench.engine.api.activityapi.core.Activity;
import io.nosqlbench.engine.api.activityapi.core.RunState;
import io.nosqlbench.engine.api.activityapi.core.progress.ProgressCapable;
import io.nosqlbench.engine.api.activityapi.core.progress.ProgressMeterDisplay;
import io.nosqlbench.engine.core.lifecycle.ExecutionResult;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
public class ActivityRuntimeInfo implements ProgressCapable {
private final static Logger logger = LogManager.getLogger(ActivityRuntimeInfo.class);
private final Activity activity;
private final Future<ExecutionResult> future;
private final ActivityExecutor executor;
private ExecutionResult result;
public ActivityRuntimeInfo(Activity activity, Future<ExecutionResult> result, ActivityExecutor executor) {
this.activity = activity;
this.future = result;
this.executor = executor;
}
@Override
public ProgressMeterDisplay getProgressMeter() {
return this.activity.getProgressMeter();
}
public Future<ExecutionResult> getFuture() {
return this.future;
}
/**
* Wait until the execution is complete and return the result.
* @param timeoutMillis
* @return null, or an ExecutionResult if the execution completed
*/
public ExecutionResult awaitResult(long timeoutMillis) {
ExecutionResult result = null;
try {
result = future.get(timeoutMillis, TimeUnit.MILLISECONDS);
} catch (TimeoutException te) {
} catch (InterruptedException ie) {
logger.warn("interrupted waiting for execution to complete");
} catch (ExecutionException e) {
throw new RuntimeException(e);
// return new ExecutionResult(activity.getStartedAtMillis(),System.currentTimeMillis(),"",e);
}
return result;
}
public Activity getActivity() {
return this.activity;
}
public boolean isRunning() {
return executor.isRunning();
}
public RunState getRunState() {
return this.activity.getRunState();
}
public void stopActivity() {
this.executor.stopActivity();
}
public ActivityExecutor getActivityExecutor() {
return executor;
}
}

View File

@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.nosqlbench.engine.core.script;
package io.nosqlbench.engine.core.lifecycle.scenario;
import com.codahale.metrics.MetricRegistry;
import com.oracle.truffle.js.scriptengine.GraalJSScriptEngine;
@ -26,11 +26,14 @@ import io.nosqlbench.api.metadata.SystemId;
import io.nosqlbench.engine.api.extensions.ScriptingPluginInfo;
import io.nosqlbench.engine.api.scripting.ScriptEnvBuffer;
import io.nosqlbench.engine.core.annotation.Annotators;
import io.nosqlbench.engine.core.lifecycle.ActivityProgressIndicator;
import io.nosqlbench.engine.core.lifecycle.ExecMetricsResult;
import io.nosqlbench.engine.core.lifecycle.PolyglotScenarioController;
import io.nosqlbench.engine.core.lifecycle.ScenarioController;
import io.nosqlbench.engine.core.metrics.PolyglotMetricRegistryBindings;
import io.nosqlbench.engine.core.lifecycle.activity.ActivityProgressIndicator;
import io.nosqlbench.engine.core.lifecycle.ExecutionMetricsResult;
import io.nosqlbench.engine.core.lifecycle.scenario.script.bindings.PolyglotScenarioController;
import io.nosqlbench.engine.core.lifecycle.scenario.script.bindings.ActivityBindings;
import io.nosqlbench.engine.core.lifecycle.scenario.script.SandboxExtensionFinder;
import io.nosqlbench.engine.core.lifecycle.scenario.script.ScenarioContext;
import io.nosqlbench.engine.core.lifecycle.scenario.script.ScriptParams;
import io.nosqlbench.engine.core.lifecycle.scenario.script.bindings.PolyglotMetricRegistryBindings;
import io.nosqlbench.nb.annotations.Maturity;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@ -58,7 +61,7 @@ import java.util.Optional;
import java.util.concurrent.Callable;
import java.util.stream.Collectors;
public class Scenario implements Callable<ExecMetricsResult> {
public class Scenario implements Callable<ExecutionMetricsResult> {
private final String commandLine;
private final String reportSummaryTo;
@ -71,9 +74,9 @@ public class Scenario implements Callable<ExecMetricsResult> {
private Exception error;
private ScenarioMetadata scenarioMetadata;
private ExecMetricsResult result;
private ExecutionMetricsResult result;
public Optional<ExecMetricsResult> getResultIfComplete() {
public Optional<ExecutionMetricsResult> getResultIfComplete() {
return Optional.ofNullable(this.result);
}
@ -213,7 +216,7 @@ public class Scenario implements Callable<ExecMetricsResult> {
scriptEngine.put("scenario", new PolyglotScenarioController(scenarioController));
scriptEngine.put("metrics", new PolyglotMetricRegistryBindings(metricRegistry));
scriptEngine.put("activities", new NashornActivityBindings(scenarioController));
scriptEngine.put("activities", new ActivityBindings(scenarioController));
for (ScriptingPluginInfo<?> extensionDescriptor : SandboxExtensionFinder.findAll()) {
if (!extensionDescriptor.isAutoLoading()) {
@ -263,20 +266,26 @@ public class Scenario implements Callable<ExecMetricsResult> {
);
logger.debug("Running control script for " + getScenarioName() + ".");
scenarioController = new ScenarioController(this.scenarioName, minMaturity);
initializeScriptingEngine(scenarioController);
executeScenarioScripts();
long awaitCompletionTime = 86400 * 365 * 1000L;
logger.debug("Awaiting completion of scenario and activities for " + awaitCompletionTime + " millis.");
scenarioController.awaitCompletion(awaitCompletionTime);
//TODO: Ensure control flow covers controller shutdown in event of internal error.
scenarioController = new ScenarioController(this,minMaturity);
try {
initializeScriptingEngine(scenarioController);
executeScenarioScripts();
long awaitCompletionTime = 86400 * 365 * 1000L;
logger.debug("Awaiting completion of scenario and activities for " + awaitCompletionTime + " millis.");
scenarioController.awaitCompletion(awaitCompletionTime);
} catch (Exception e) {
this.error=e;
} finally {
scenarioController.shutdown();
}
Runtime.getRuntime().removeShutdownHook(scenarioShutdownHook);
scenarioShutdownHook.run();
}
public void notifyException(Thread t, Throwable e) {
this.error=new RuntimeException("in thread " + t.getName() + ", " +e, e);
}
private void executeScenarioScripts() {
for (String script : scripts) {
try {
@ -375,35 +384,32 @@ public class Scenario implements Callable<ExecMetricsResult> {
*
* The lifecycle of a scenario includes the lifecycles of all of the following:
* <OL>
* <LI>The scenario control script, executing within a graaljs context.</LI>
* <LI>The lifecycle of every activity which is started within the scenario.</LI>
* <LI>The scenario control script, executing within a graaljs context.</LI>
* <LI>The lifecycle of every activity which is started within the scenario.</LI>
* </OL>
*
* All of these run asynchronously within the scenario, however the same thread that calls
* the scenario is the one which executes the control script. A scenario ends when all
* of the following conditions are met:
* <UL>
* <LI>The scenario control script has run to completion, or experienced an exception.</LI>
* <LI>Each activity has run to completion, experienced an exception, or all</LI>
* <LI>The scenario control script has run to completion, or experienced an exception.</LI>
* <LI>Each activity has run to completion, experienced an exception, or all</LI>
* </UL>
*
* @return
*/
public synchronized ExecMetricsResult call() {
public synchronized ExecutionMetricsResult call() {
if (result == null) {
try {
runScenario();
} catch (Exception e) {
if (this.error!=null) {
logger.debug("OVERLAPPING ERRORS: prior" + this.error.getMessage() + ", current:" + e.getMessage());
}
this.error = e;
this.error=e;
} finally {
logger.debug((this.error == null ? "NORMAL" : "ERRORED") + " scenario run");
logger.debug((this.error==null ? "NORMAL" : "ERRORED") + " scenario run");
}
String iolog = scriptEnv.getTimedLog();
this.result = new ExecMetricsResult(this.error, iolog, this.startedAtMillis, this.endedAtMillis);
this.result = new ExecutionMetricsResult(this.startedAtMillis, this.endedAtMillis, iolog, error);
result.reportMetricsSummaryToLog();
doReportSummaries(reportSummaryTo, result);
}
@ -411,7 +417,7 @@ public class Scenario implements Callable<ExecMetricsResult> {
return result;
}
private void doReportSummaries(String reportSummaryTo, ExecMetricsResult result) {
private void doReportSummaries(String reportSummaryTo, ExecutionMetricsResult result) {
List<PrintStream> fullChannels = new ArrayList<>();
List<PrintStream> briefChannels = new ArrayList<>();

View File

@ -70,7 +70,7 @@ public class ScenarioController {
*/
public synchronized void start(ActivityDef activityDef) {
Annotators.recordAnnotation(Annotation.newBuilder()
.session(sessionId)
.session(scenario.getScenarioName())
.now()
.layer(Layer.Activity)
.label("alias", activityDef.getAlias())
@ -129,7 +129,7 @@ public class ScenarioController {
*/
public synchronized void run(ActivityDef activityDef, long timeoutMs) {
Annotators.recordAnnotation(Annotation.newBuilder()
.session(sessionId)
.session(this.scenario.getScenarioName())
.now()
.layer(Layer.Activity)
.label("alias", activityDef.getAlias())
@ -165,9 +165,8 @@ public class ScenarioController {
}
public boolean isRunningActivity(ActivityDef activityDef) {
ActivityThreadsManager activityThreadsManager = getActivityExecutor(activityDef, false);
return activityThreadsManager != null && activityThreadsManager.isRunning();
ActivityRuntimeInfo runtimeInfo = this.activityInfoMap.get(activityDef.getAlias());
return (runtimeInfo != null && runtimeInfo.isRunning());
}
public boolean isRunningActivity(Map<String, String> activityDefMap) {
@ -184,7 +183,7 @@ public class ScenarioController {
*/
public synchronized void stop(ActivityDef activityDef) {
Annotators.recordAnnotation(Annotation.newBuilder()
.session(sessionId)
.session(this.scenario.getScenarioName())
.now()
.layer(Layer.Activity)
.label("alias", activityDef.getAlias())
@ -192,18 +191,14 @@ public class ScenarioController {
.detail("params", activityDef.toString())
.build());
ActivityThreadsManager activityThreadsManager = getActivityExecutor(activityDef, false);
if (activityThreadsManager == null) {
ActivityRuntimeInfo runtimeInfo = this.activityInfoMap.get(activityDef.getAlias());
if (runtimeInfo == null) {
throw new RuntimeException("could not stop missing activity:" + activityDef);
}
RunState runstate = activityThreadsManager.getActivity().getRunState();
if (runstate != RunState.Running) {
logger.warn("NOT stopping activity '" + activityThreadsManager.getActivity().getAlias() + "' because it is in state '" + runstate + "'");
return;
}
scenariologger.debug("STOP " + activityDef.getAlias());
activityThreadsManager.stopActivity();
runtimeInfo.stopActivity();
}
/**
@ -240,76 +235,6 @@ public class ScenarioController {
}
}
/**
* Modify one of the parameters in a defined activity. Any observing activity components will be notified of the
* changes made to activity parameters.
*
* @param alias The name of an activity that is already known to the scenario.
* @param param The parameter name
* @param value a new parameter value
*/
public synchronized void modify(String alias, String param, String value) {
if (param.equals("alias")) {
throw new InvalidParameterException("It is not allowed to change the name of an existing activity.");
}
ActivityThreadsManager activityThreadsManager = getActivityExecutor(alias);
ParameterMap params = activityThreadsManager.getActivityDef().getParams();
scenariologger.debug("SET (" + alias + "/" + param + ")=(" + value + ")");
params.set(param, value);
}
/**
* Apply any parameter changes to a defined activity, or start a new one.
* This method is syntactical sugar for scripting. Each of the parameters in the map
* is checked against existing values, and per-field modifications
* are applied one at a time, only if the values have changed.
*
* @param appliedParams Map of new values.
*/
public synchronized void apply(Map<String, String> appliedParams) {
String alias = appliedParams.get("alias");
if (alias == null) {
throw new BasicError("alias must be provided");
}
ActivityThreadsManager executor = activityExecutors.get(alias);
if (executor == null) {
logger.info("started scenario from apply:" + alias);
start(appliedParams);
return;
}
ParameterMap previousMap = executor.getActivityDef().getParams();
for (String paramName : appliedParams.keySet()) {
String appliedVal = appliedParams.get(paramName);
Optional<String> prevVal = previousMap.getOptionalString(paramName);
if (!prevVal.isPresent() || !prevVal.get().equals(appliedVal)) {
logger.info("applying new value to activity '" + alias + "': '" + prevVal.get() + "' -> '" + appliedVal + "'");
previousMap.set(paramName, appliedVal);
}
}
}
/**
* Get the activity executor associated with the given alias. This should be used to find activitytypes
* which are presumed to be already defined.
*
* @param activityAlias The activity alias for the extant activity.
* @return the associated ActivityExecutor
* @throws RuntimeException a runtime exception if the named activity is not found
*/
private ActivityThreadsManager getActivityExecutor(String activityAlias) {
Optional<ActivityThreadsManager> executor =
Optional.ofNullable(activityExecutors.get(activityAlias));
return executor.orElseThrow(
() -> new RuntimeException("ActivityExecutor for alias " + activityAlias + " not found.")
);
}
private List<String> getMatchingAliases(String pattern) {
Pattern matcher;
@ -320,52 +245,13 @@ public class ScenarioController {
matcher = Pattern.compile(pattern);
}
List<String> matching = activityExecutors.keySet().stream()
List<String> matching = activityInfoMap.keySet().stream()
.filter(a -> Pattern.matches(pattern, a))
.peek(p -> logger.debug("MATCH " + pattern + " -> " + p))
.collect(Collectors.toList());
return matching;
}
private ActivityThreadsManager getActivityExecutor(ActivityDef activityDef, boolean createIfMissing) {
synchronized (activityExecutors) {
ActivityThreadsManager executor = activityExecutors.get(activityDef.getAlias());
if (executor == null && createIfMissing) {
if (activityDef.getParams().containsKey("driver")) {
ActivityType<?> activityType = new ActivityTypeLoader()
.setMaturity(this.minMaturity)
.load(activityDef)
.orElseThrow(
() -> new RuntimeException("Driver for '" + activityDef + "' was not found." +
"\nYou can use --list-drivers to see what drivers are supported in this runtime." +
ConfigSuggestions.suggestAlternates(
new ActivityTypeLoader().getAllSelectors(), activityDef.getActivityType(), 4)
.orElse("")
)
);
executor = new ActivityThreadsManager(
activityType.getAssembledActivity(
activityDef,
getActivityMap()
),
this.sessionId
);
activityExecutors.put(activityDef.getAlias(), executor);
} else {
executor = new ActivityThreadsManager(
new StandardActivityType(activityDef).getAssembledActivity(
activityDef, getActivityMap()
), this.sessionId
);
}
}
return executor;
}
}
/**
* Wait for a bit. This is not the best approach, and will be replaced with a different system in the future.
*
@ -394,29 +280,7 @@ public class ScenarioController {
* @return set of activity names
*/
public Set<String> getAliases() {
return activityExecutors.keySet();
}
/**
* Return all the activity definitions that are known to this scenario.
*
* @return list of activity defs
*/
public List<ActivityDef> getActivityDefs() {
return activityExecutors.values().stream()
.map(ActivityThreadsManager::getActivityDef)
.collect(Collectors.toList());
}
/**
* Get the named activity def, if it is known to this scenario.
*
* @param alias The name by which the activity is known to this scenario.
* @return an ActivityDef instance
* @throws RuntimeException if the alias is not known to the scenario
*/
public ActivityDef getActivityDef(String alias) {
return getActivityExecutor(alias).getActivityDef();
return activityInfoMap.keySet();
}
/**
@ -427,8 +291,9 @@ public class ScenarioController {
* @param waitTimeMillis grace period during which an activity may cooperatively shut down
*/
public synchronized void forceStopScenario(int waitTimeMillis, boolean rethrow) {
logger.debug("force stopping scenario " + this.scenario.getScenarioName());
activityInfoMap.values().forEach(a -> a.getActivityExecutor().forceStopActivity(10000));
logger.debug("Scenario force stopped.");
activityExecutors.values().forEach(a -> a.forceStopScenarioAndThrow(waitTimeMillis, rethrow));
}
// public synchronized void stopAll() {
@ -446,31 +311,21 @@ public class ScenarioController {
public boolean awaitCompletion(long waitTimeMillis) {
logger.debug(() -> "awaiting completion");
boolean completed = true;
long remaining = waitTimeMillis;
List<ActivityFinisher> finishers = new ArrayList<>();
for (ActivityThreadsManager ae : activityExecutors.values()) {
ActivityFinisher finisher = new ActivityFinisher(ae, (int) remaining);
finishers.add(finisher);
finisher.start();
}
for (ActivityFinisher finisher : finishers) {
try {
logger.debug("joining finisher " + finisher.getName());
finisher.join(waitTimeMillis);
logger.debug("joined finisher " + finisher.getName());
} catch (InterruptedException ignored) {
}
}
for (ActivityFinisher finisher : finishers) {
if (!finisher.getResult()) {
logger.debug("finisher for " + finisher.getName() + " did not signal TRUE");
for (ActivityRuntimeInfo activityRuntimeInfo : this.activityInfoMap.values()) {
ExecutionResult activityResult = activityRuntimeInfo.awaitResult(waitTimeMillis);
if (activityResult == null) {
logger.error("Unable to retrieve activity result for " + activityRuntimeInfo.getActivity().getAlias());
completed = false;
} else {
if (activityResult.getException()!=null) {
if (activityResult.getException() instanceof RuntimeException e) {
throw e;
} else {
throw new RuntimeException(activityResult.getException());
}
}
}
}
return completed;
}
@ -482,66 +337,98 @@ public class ScenarioController {
}
}
public boolean await(Map<String, String> activityDefMap) {
return this.awaitActivity(activityDefMap);
public void await(Map<String, String> activityDefMap) {
this.awaitActivity(activityDefMap);
}
public boolean awaitActivity(Map<String, String> activityDefMap) {
ActivityDef ad = new ActivityDef(new ParameterMap(activityDefMap));
return awaitActivity(ad);
return awaitActivity(ad, Long.MAX_VALUE);
}
public boolean await(String alias) {
return this.awaitActivity(alias);
return this.awaitActivity(alias, Long.MAX_VALUE);
}
public boolean awaitActivity(String alias) {
public boolean awaitActivity(String alias, long timeoutMs) {
ActivityDef toAwait = aliasToDef(alias);
return awaitActivity(toAwait);
return awaitActivity(toAwait, Long.MAX_VALUE);
}
public boolean await(ActivityDef activityDef) {
return this.awaitActivity(activityDef);
public void await(ActivityDef activityDef, long timeoutMs) {
this.awaitActivity(activityDef, timeoutMs);
}
public boolean awaitActivity(ActivityDef activityDef) {
ActivityThreadsManager activityThreadsManager = getActivityExecutor(activityDef, false);
if (activityThreadsManager == null) {
throw new RuntimeException("Could not await missing activity: " + activityDef);
public boolean awaitActivity(ActivityDef activityDef, long timeoutMs) {
ActivityRuntimeInfo ari = this.activityInfoMap.get(activityDef.getAlias());
if (ari == null) {
throw new RuntimeException("Could not await missing activity: " + activityDef.getAlias());
}
scenariologger.debug("AWAIT/before alias=" + activityDef.getAlias());
boolean finished = activityThreadsManager.awaitFinishedOrStopped(Integer.MAX_VALUE);
scenariologger.debug("AWAIT/after completed=" + finished);
return finished;
ExecutionResult result = null;
Future<ExecutionResult> future=null;
try {
future = ari.getFuture();
} catch (Exception e) {
throw new RuntimeException(e);
}
try {
result = future.get(timeoutMs, TimeUnit.MILLISECONDS);
} catch (ExecutionException e) {
throw new RuntimeException(e);
} catch (InterruptedException e) {
throw new RuntimeException(e);
} catch (TimeoutException e) {
throw new RuntimeException(e);
}
return (result != null);
}
/**
* @return an unmodifyable String to executor map of all activities known to this scenario
*/
public Map<String, ActivityThreadsManager> getActivityExecutorMap() {
return Collections.unmodifiableMap(activityExecutors);
public Map<String, ActivityRuntimeInfo> getActivityExecutorMap() {
return Collections.unmodifiableMap(activityInfoMap);
}
public List<ActivityDef> getActivityDefs() {
return activityInfoMap.values().stream().map(ari -> ari.getActivity().getActivityDef()).toList();
}
public void reportMetrics() {
ActivityMetrics.reportTo(System.out);
}
private Map<String, Activity> getActivityMap() {
Map<String, Activity> activityMap = new HashMap<String, Activity>();
for (Map.Entry<String, ActivityThreadsManager> entry : activityExecutors.entrySet()) {
activityMap.put(entry.getKey(), entry.getValue().getActivity());
}
return activityMap;
}
public List<ProgressMeterDisplay> getProgressMeters() {
List<ProgressMeterDisplay> indicators = new ArrayList<>();
for (ActivityThreadsManager ae : activityExecutors.values()) {
for (ActivityRuntimeInfo ae : activityInfoMap.values()) {
indicators.add(ae.getProgressMeter());
}
indicators.sort(Comparator.comparing(ProgressMeterDisplay::getStartTime));
return indicators;
}
public void notifyException(Thread t, Throwable e) {
logger.error("Uncaught exception in activity lifecycle thread:" + e, e);
scenario.notifyException(t,e);
throw new RuntimeException(e);
}
public ActivityDef getActivityDef(String alias) {
return activityInfoMap.get(alias).getActivity().getActivityDef();
}
public void shutdown() {
this.activitiesExecutor.shutdown();
try {
if (!this.activitiesExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
this.activitiesExecutor.shutdownNow();
if (!this.activitiesExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
throw new RuntimeException("Unable to shutdown activities executor");
}
}
} catch (Exception e) {
}
}
}

View File

@ -79,6 +79,7 @@ public class ScenariosExecutor {
* @return the final scenario-result map
*/
public ScenariosResults awaitAllResults(long timeout, long updateInterval) {
long waitFrom = System.currentTimeMillis();
if (updateInterval > timeout) {
throw new BasicError("timeout must be equal to or greater than updateInterval");
}
@ -98,6 +99,7 @@ public class ScenariosExecutor {
} catch (InterruptedException ignored) {
}
}
logger.trace("waited " + (System.currentTimeMillis()-waitFrom) + " millis for scenarios");
updateAt = Math.min(timeoutAt, System.currentTimeMillis() + updateInterval);
}
@ -193,10 +195,7 @@ public class ScenariosExecutor {
logger.debug("#stopScenario(name=" + scenarioName + ", rethrow="+ rethrow+")");
Optional<Scenario> pendingScenario = getPendingScenario(scenarioName);
if (pendingScenario.isPresent()) {
ScenarioController controller = pendingScenario.get().getScenarioController();
if (controller != null) {
controller.forceStopScenario(0, rethrow);
}
pendingScenario.get().getScenarioController().forceStopScenario(10000, true);
} else {
throw new RuntimeException("Unable to cancel scenario: " + scenarioName + ": not found");
}

View File

@ -41,8 +41,8 @@ public class ScenariosResults {
public String getExecutionSummary() {
String sb = "executions: " + scenarioResultMap.size() + " scenarios, " +
scenarioResultMap.values().stream().filter(r -> r.getException().isEmpty()).count() + " normal, " +
scenarioResultMap.values().stream().filter(r -> r.getException().isPresent()).count() + " errored";
scenarioResultMap.values().stream().filter(r -> r.getException()==null).count() + " normal, " +
scenarioResultMap.values().stream().filter(r -> r.getException()!=null).count() + " errored";
return sb;
}
@ -72,7 +72,7 @@ public class ScenariosResults {
public boolean hasError() {
return this.scenarioResultMap.values().stream()
.anyMatch(r -> r.getException().isPresent());
.anyMatch(r -> r.getException()!=null);
}
public int getSize() {

View File

@ -16,30 +16,35 @@
package io.nosqlbench.engine.core;
import io.nosqlbench.api.engine.activityimpl.ActivityDef;
import io.nosqlbench.engine.api.activityapi.core.*;
import io.nosqlbench.engine.api.activityapi.output.OutputDispenser;
import io.nosqlbench.engine.api.activityapi.input.Input;
import io.nosqlbench.engine.api.activityapi.input.InputDispenser;
import io.nosqlbench.api.engine.activityimpl.ActivityDef;
import io.nosqlbench.engine.api.activityapi.output.OutputDispenser;
import io.nosqlbench.engine.api.activityimpl.CoreServices;
import io.nosqlbench.engine.api.activityimpl.SimpleActivity;
import io.nosqlbench.engine.api.activityimpl.action.CoreActionDispenser;
import io.nosqlbench.engine.api.activityimpl.input.CoreInputDispenser;
import io.nosqlbench.engine.api.activityimpl.input.AtomicInput;
import io.nosqlbench.engine.api.activityimpl.input.CoreInputDispenser;
import io.nosqlbench.engine.api.activityimpl.motor.CoreMotor;
import io.nosqlbench.engine.api.activityimpl.motor.CoreMotorDispenser;
import io.nosqlbench.engine.core.lifecycle.ActivityThreadsManager;
import io.nosqlbench.engine.core.lifecycle.ActivityTypeLoader;
import org.apache.logging.log4j.Logger;
import io.nosqlbench.engine.core.lifecycle.ExecutionResult;
import io.nosqlbench.engine.core.lifecycle.activity.ActivityExecutor;
import io.nosqlbench.engine.core.lifecycle.activity.ActivityTypeLoader;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.junit.jupiter.api.Test;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import static org.assertj.core.api.Assertions.assertThat;
public class ActivityThreadsManagerTest {
private static final Logger logger = LogManager.getLogger(ActivityThreadsManagerTest.class);
public class ActivityExecutorTest {
private static final Logger logger = LogManager.getLogger(ActivityExecutorTest.class);
@Test
public synchronized void testRestart() {
@ -54,13 +59,30 @@ public class ActivityThreadsManagerTest {
a.setOutputDispenserDelegate(tdisp);
a.setInputDispenserDelegate(idisp);
a.setMotorDispenserDelegate(mdisp);
ActivityThreadsManager ae = new ActivityThreadsManager(a, "test-restart");
ad.setThreads(1);
ae.startActivity();
ae.stopActivity();
ae.startActivity();
ae.awaitCompletion(15000);
ExecutorService executor = Executors.newCachedThreadPool();
ActivityExecutor ae = new ActivityExecutor(a, "test-restart");
Future<ExecutionResult> future = executor.submit(ae);
try {
System.out.println("ad.setThreads(1)");
ad.setThreads(1);
System.out.println("ae.startActivity()");
ae.startActivity();
System.out.println("ae.stopActivity()");
ae.stopActivity();
System.out.println("ae.startActivity()");
ae.startActivity();
System.out.println("ae.startActivity()");
ae.startActivity();
System.out.println("ExecutionResult executionResult = future.get();");
ExecutionResult executionResult = future.get();
System.out.println("System.out.print(executionResult);");
System.out.print(executionResult);
Thread.sleep(500L);
} catch (Exception e) {
throw new RuntimeException(e);
}
System.out.print("ad.setThreads(1)");
executor.shutdown();
assertThat(idisp.getInput(10).getInputSegment(3)).isNull();
}
@ -79,10 +101,20 @@ public class ActivityThreadsManagerTest {
a.setInputDispenserDelegate(idisp);
a.setMotorDispenserDelegate(mdisp);
ActivityThreadsManager ae = new ActivityThreadsManager(a, "test-delayed-start");
ad.setThreads(1);
ae.startActivity();
ae.awaitCompletion(15000);
ActivityExecutor ae = new ActivityExecutor(a, "test-delayed-start");
ExecutorService testExecutor = Executors.newCachedThreadPool();
Future<ExecutionResult> future = testExecutor.submit(ae);
try {
ad.setThreads(1);
ae.startActivity();
ExecutionResult result = future.get();
} catch (InterruptedException e) {
throw new RuntimeException(e);
} catch (ExecutionException e) {
throw new RuntimeException(e);
}
testExecutor.shutdownNow();
assertThat(idisp.getInput(10).getInputSegment(3)).isNull();
}
@ -93,7 +125,7 @@ public class ActivityThreadsManagerTest {
Optional<ActivityType> activityType = new ActivityTypeLoader().load(ad);
Input longSupplier = new AtomicInput(ad);
MotorDispenser<?> cmf = getActivityMotorFactory(
ad, motorActionDelay(999), longSupplier
ad, motorActionDelay(999), longSupplier
);
Activity a = new SimpleActivity(ad);
InputDispenser idisp = new CoreInputDispenser(a);
@ -104,15 +136,15 @@ public class ActivityThreadsManagerTest {
a.setInputDispenserDelegate(idisp);
a.setMotorDispenserDelegate(mdisp);
ActivityThreadsManager ae = new ActivityThreadsManager(a, "test-new-executor");
ActivityExecutor ae = new ActivityExecutor(a, "test-new-executor");
ad.setThreads(5);
ae.startActivity();
int[] speeds = new int[]{1,2000,5,2000,2,2000};
for(int offset=0; offset<speeds.length; offset+=2) {
int threadTarget=speeds[offset];
int threadTime = speeds[offset+1];
logger.info("Setting thread level to " + threadTarget + " for " +threadTime + " seconds.");
int[] speeds = new int[]{1, 2000, 5, 2000, 2, 2000};
for (int offset = 0; offset < speeds.length; offset += 2) {
int threadTarget = speeds[offset];
int threadTime = speeds[offset + 1];
logger.info("Setting thread level to " + threadTarget + " for " + threadTime + " seconds.");
ad.setThreads(threadTarget);
try {
Thread.sleep(threadTime);