runtime improvements related to scheduling points (#293)

This commit is contained in:
Pantazis Deligiannis 2022-03-08 13:27:44 -08:00 коммит произвёл GitHub
Родитель 3f4ef590ec
Коммит 37d7636e65
Не найден ключ, соответствующий данной подписи
Идентификатор ключа GPG: 4AEE18F83AFDEB23
17 изменённых файлов: 329 добавлений и 254 удалений

Просмотреть файл

@ -576,15 +576,6 @@ namespace Microsoft.Coyote.Actors
{
}
/// <summary>
/// Notifies that the inbox of the specified actor is about to be
/// checked to see if the default event handler should fire.
/// </summary>
[MethodImpl(MethodImplOptions.AggressiveInlining)]
internal virtual void LogDefaultEventHandlerCheck(Actor actor)
{
}
/// <summary>
/// Logs that the specified actor raised an <see cref="Event"/>.
/// </summary>
@ -1386,11 +1377,6 @@ namespace Microsoft.Coyote.Actors
this.ResetProgramCounter(actor);
}
/// <inheritdoc/>
[MethodImpl(MethodImplOptions.AggressiveInlining)]
internal override void LogDefaultEventHandlerCheck(Actor actor) =>
this.Runtime.ScheduleNextOperation(SchedulingPointType.Default);
/// <inheritdoc/>
internal override void LogRaisedEvent(Actor actor, Event e, EventGroup eventGroup, EventInfo eventInfo)
{
@ -1446,7 +1432,7 @@ namespace Microsoft.Coyote.Actors
this.LogWriter.LogWaitEvent(actor.Id, stateName, eventWaitTypesArray);
}
this.Runtime.ScheduleNextOperation(SchedulingPointType.Join);
this.Runtime.ScheduleNextOperation(SchedulingPointType.Wait);
this.ResetProgramCounter(actor);
}

Просмотреть файл

@ -318,7 +318,7 @@ namespace Microsoft.Coyote.Actors.Mocks
bool result = this.Owner.IsDefaultHandlerInstalled();
if (result)
{
this.Owner.Context.Runtime.ScheduleNextOperation(Runtime.SchedulingPointType.Default);
this.Owner.Context.Runtime.ScheduleNextOperation(Runtime.SchedulingPointType.Receive);
}
return result;

Просмотреть файл

@ -58,10 +58,8 @@ namespace Microsoft.Coyote.Runtime
/// <summary>
/// The runtime executing the current operation.
/// </summary>
internal static CoyoteRuntime Current
{
get => ThreadLocalRuntime.Value ?? AsyncLocalRuntime.Value ?? RuntimeProvider.DefaultRuntime;
}
internal static CoyoteRuntime Current =>
ThreadLocalRuntime.Value ?? AsyncLocalRuntime.Value ?? RuntimeProvider.DefaultRuntime;
/// <summary>
/// If true, the program execution is controlled by the runtime to
@ -144,11 +142,6 @@ namespace Microsoft.Coyote.Runtime
/// </summary>
private readonly HashSet<string> UncontrolledInvocations;
/// <summary>
/// The program schedule trace.
/// </summary>
internal readonly ScheduleTrace ScheduleTrace;
/// <summary>
/// The currently scheduled operation during systematic testing.
/// </summary>
@ -185,13 +178,14 @@ namespace Microsoft.Coyote.Runtime
private bool IsSchedulerSuppressed;
/// <summary>
/// True if the last scheduling point was postponed. This happens if an uncontrolled
/// thread created a new controlled operation and tried to schedule it, but this can
/// only happen from a controlled thread. If this value is set, the runtime will try
/// to invoke the scheduler from a controlled thread before resuming executing the
/// currently scheduled operation, which can potentially increase coverage.
/// If this value is not null, then it represents the last scheduling point that
/// was postponed. This happens if an uncontrolled thread created a new controlled
/// operation and tried to schedule it, but this can only happen from a controlled
/// thread. If this value is set, the runtime will try to invoke the scheduler from
/// a controlled thread before resuming executing the currently scheduled operation,
/// which can potentially increase coverage.
/// </summary>
private bool IsLastSchedulingPointPostponed;
private SchedulingPointType? LastPostponedSchedulingPoint;
/// <summary>
/// True if uncontrolled concurrency was detected, else false.
@ -281,7 +275,7 @@ namespace Microsoft.Coyote.Runtime
this.IsAttached = true;
this.IsSchedulerSuppressed = false;
this.IsUncontrolledConcurrencyDetected = false;
this.IsLastSchedulingPointPostponed = false;
this.LastPostponedSchedulingPoint = null;
this.MaxConcurrencyDegree = 0;
this.IsBugFound = false;
@ -292,7 +286,6 @@ namespace Microsoft.Coyote.Runtime
this.UncontrolledTasks = new HashSet<Task>();
this.UncontrolledInvocations = new HashSet<string>();
this.CompletionSource = new TaskCompletionSource<bool>();
this.ScheduleTrace = new ScheduleTrace();
if (this.SchedulingPolicy is SchedulingPolicy.Systematic)
{
@ -337,12 +330,8 @@ namespace Microsoft.Coyote.Runtime
{
try
{
// Install the runtime to the execution context of the current thread.
this.SetThreadExecutionContext();
// Update the controlled thread with the currently executing operation,
// allowing future retrieval in the same controlled thread.
ExecutingOperation.Value = op;
// Sets up the execution context of the current thread.
this.SetCurrentExecutionContext(op);
this.StartOperation(op);
@ -396,10 +385,7 @@ namespace Microsoft.Coyote.Runtime
}
finally
{
// Clean the thread local state.
ExecutingOperation.Value = null;
AsyncLocalRuntime.Value = null;
ThreadLocalRuntime.Value = null;
CleanCurrentExecutionContext();
}
});
@ -431,7 +417,7 @@ namespace Microsoft.Coyote.Runtime
new DelayOperation(operationId, $"Delay({operationId})", delay) :
new ControlledOperation(operationId, $"Op({operationId})");
this.RegisterOperation(op);
if (operationId > 0 && this.IsThreadUncontrolled(Thread.CurrentThread))
if (operationId > 0 && !this.IsThreadControlled(Thread.CurrentThread))
{
op.IsSourceUncontrolled = true;
}
@ -439,6 +425,63 @@ namespace Microsoft.Coyote.Runtime
return op;
}
/// <summary>
/// Schedules the specified task to be executed asynchronously.
/// </summary>
internal void Schedule(Task task)
{
lock (this.SyncObject)
{
if (!this.IsAttached)
{
return;
}
}
// Check if an existing controlled operation is stored in the state of the task.
ControlledOperation op = task.AsyncState as ControlledOperation ?? this.CreateControlledOperation();
var thread = new Thread(() =>
{
try
{
// Sets up the execution context of the current thread.
this.SetCurrentExecutionContext(op);
this.StartOperation(op);
if (this.SchedulingPolicy is SchedulingPolicy.Fuzzing)
{
this.DelayOperation();
}
this.ControlledTaskScheduler.ExecuteTask(task);
this.CompleteOperation(op);
this.ScheduleNextOperation(SchedulingPointType.Complete);
}
catch (Exception ex)
{
this.ProcessUnhandledExceptionInOperation(op, ex);
}
finally
{
CleanCurrentExecutionContext();
}
});
thread.Name = Guid.NewGuid().ToString();
thread.IsBackground = true;
// TODO: optimize by using a real threadpool instead of creating a new thread each time.
this.ThreadPool.AddOrUpdate(op.Id, thread, (id, oldThread) => thread);
this.ControlledThreads.AddOrUpdate(thread.Name, op, (threadName, oldOp) => op);
this.ControlledTasks.TryAdd(task, op);
thread.Start();
this.WaitOperationStart(op);
this.ScheduleNextOperation(SchedulingPointType.Create);
}
/// <summary>
/// Schedules the specified callback to be executed asynchronously.
/// </summary>
@ -457,15 +500,10 @@ namespace Microsoft.Coyote.Runtime
{
try
{
// Install the runtime to the execution context of the current thread.
this.SetThreadExecutionContext();
// Update the controlled thread with the currently executing operation,
// allowing future retrieval in the same controlled thread.
ExecutingOperation.Value = op;
// Sets up the execution context of the current thread.
this.SetCurrentExecutionContext(op);
this.StartOperation(op);
if (this.SchedulingPolicy is SchedulingPolicy.Fuzzing)
{
this.DelayOperation();
@ -473,7 +511,7 @@ namespace Microsoft.Coyote.Runtime
callback();
this.CompleteOperation(op);
this.ScheduleNextOperation(SchedulingPointType.Stop);
this.ScheduleNextOperation(SchedulingPointType.Complete);
}
catch (Exception ex)
{
@ -481,10 +519,7 @@ namespace Microsoft.Coyote.Runtime
}
finally
{
// Clean the thread local state.
ExecutingOperation.Value = null;
AsyncLocalRuntime.Value = null;
ThreadLocalRuntime.Value = null;
CleanCurrentExecutionContext();
}
});
@ -498,72 +533,7 @@ namespace Microsoft.Coyote.Runtime
thread.Start();
this.WaitOperationStart(op);
this.ScheduleNextOperation(SchedulingPointType.Create);
}
/// <summary>
/// Schedules the specified task to be executed asynchronously.
/// </summary>
internal void ScheduleTask(Task task)
{
lock (this.SyncObject)
{
if (!this.IsAttached)
{
return;
}
}
// Check if an existing controlled operation is stored in the state of the task.
ControlledOperation op = task.AsyncState as ControlledOperation ?? this.CreateControlledOperation();
var thread = new Thread(() =>
{
try
{
// Install the runtime to the execution context of the current thread.
this.SetThreadExecutionContext();
// Update the controlled thread with the currently executing operation,
// allowing future retrieval in the same controlled thread.
ExecutingOperation.Value = op;
this.StartOperation(op);
if (this.SchedulingPolicy is SchedulingPolicy.Fuzzing)
{
this.DelayOperation();
}
this.ControlledTaskScheduler.ExecuteTask(task);
this.CompleteOperation(op);
this.ScheduleNextOperation(SchedulingPointType.Stop);
}
catch (Exception ex)
{
this.ProcessUnhandledExceptionInOperation(op, ex);
}
finally
{
// Clean the thread local state.
ExecutingOperation.Value = null;
AsyncLocalRuntime.Value = null;
ThreadLocalRuntime.Value = null;
}
});
thread.Name = Guid.NewGuid().ToString();
thread.IsBackground = true;
// TODO: optimize by using a real threadpool instead of creating a new thread each time.
this.ThreadPool.AddOrUpdate(op.Id, thread, (id, oldThread) => thread);
this.ControlledThreads.AddOrUpdate(thread.Name, op, (threadName, oldOp) => op);
this.ControlledTasks.TryAdd(task, op);
thread.Start();
this.WaitOperationStart(op);
this.ScheduleNextOperation(SchedulingPointType.Create);
this.ScheduleNextOperation(SchedulingPointType.ContinueWith);
}
/// <summary>
@ -604,7 +574,7 @@ namespace Microsoft.Coyote.Runtime
this.TaskFactory.Scheduler);
}
var current = this.GetExecutingOperation<ControlledOperation>();
var current = this.GetExecutingOperation();
if (current is null)
{
// Cannot fuzz the delay of an uncontrolled operation.
@ -631,7 +601,7 @@ namespace Microsoft.Coyote.Runtime
return true;
}
var callerOp = this.GetExecutingOperation<ControlledOperation>();
var callerOp = this.GetExecutingOperation();
this.WaitUntilTasksComplete(callerOp, tasks, waitAll: true);
// TODO: support timeouts during testing, this would become false if there is a timeout.
@ -657,7 +627,7 @@ namespace Microsoft.Coyote.Runtime
throw new ArgumentException("The tasks argument contains no tasks.");
}
var callerOp = this.GetExecutingOperation<ControlledOperation>();
var callerOp = this.GetExecutingOperation();
this.WaitUntilTasksComplete(callerOp, tasks, waitAll: false);
int result = -1;
@ -699,7 +669,7 @@ namespace Microsoft.Coyote.Runtime
if (op.Dependencies.Count > 0)
{
op.Status = waitAll ? OperationStatus.BlockedOnWaitAll : OperationStatus.BlockedOnWaitAny;
this.ScheduleNextOperation(SchedulingPointType.Join);
this.ScheduleNextOperation(SchedulingPointType.Wait);
}
}
}
@ -720,7 +690,7 @@ namespace Microsoft.Coyote.Runtime
this.NotifyUncontrolledTaskWait(task);
}
var op = this.GetExecutingOperation<ControlledOperation>();
var op = this.GetExecutingOperation();
this.WaitUntilTaskCompletes(op, task);
}
else if (this.SchedulingPolicy is SchedulingPolicy.Fuzzing)
@ -740,7 +710,7 @@ namespace Microsoft.Coyote.Runtime
IO.Debug.WriteLine("<CoyoteDebug> Operation '{0}' is waiting for task '{1}'.", op.Id, task.Id);
op.SetDependency(task, this.ControlledTasks.ContainsKey(task));
op.Status = OperationStatus.BlockedOnWaitAll;
this.ScheduleNextOperation(SchedulingPointType.Join);
this.ScheduleNextOperation(SchedulingPointType.Wait);
}
}
@ -814,20 +784,20 @@ namespace Microsoft.Coyote.Runtime
return;
}
if (type is SchedulingPointType.Create)
if (type is SchedulingPointType.Create || type is SchedulingPointType.ContinueWith)
{
// If this scheduling point was triggered because a new operation was created by
// an uncontrolled thread, then the scheduling point must be postponed.
if (this.IsThreadUncontrolled(Thread.CurrentThread))
if (!this.IsThreadControlled(Thread.CurrentThread))
{
IO.Debug.WriteLine("<CoyoteDebug> Postponing scheduling point in uncontrolled thread '{0}'.",
Thread.CurrentThread.ManagedThreadId);
this.IsLastSchedulingPointPostponed = true;
IO.Debug.WriteLine("<CoyoteDebug> Postponing scheduling point '{0}' in uncontrolled thread '{1}'.",
type, Thread.CurrentThread.ManagedThreadId);
this.LastPostponedSchedulingPoint = type;
return;
}
}
var current = this.GetExecutingOperation<ControlledOperation>();
var current = this.GetExecutingOperation();
if (current is null)
{
// Cannot schedule the next operation if there is no controlled operation
@ -842,7 +812,7 @@ namespace Microsoft.Coyote.Runtime
return;
}
if (this.IsSchedulerSuppressed && !this.IsLastSchedulingPointPostponed &&
if (this.IsSchedulerSuppressed && this.LastPostponedSchedulingPoint is null &&
isSuppressible && current.Status is OperationStatus.Enabled)
{
// Suppress the scheduling point.
@ -854,13 +824,13 @@ namespace Microsoft.Coyote.Runtime
this.CheckIfSchedulingStepsBoundIsReached();
// Update metadata related to this scheduling point.
current.SchedulingPoint = type;
this.IsLastSchedulingPointPostponed = false;
current.LastSchedulingPoint = type;
this.LastPostponedSchedulingPoint = null;
if (this.Configuration.IsProgramStateHashingEnabled)
{
// Update the current operation with the hashed program state.
current.HashedProgramState = this.GetHashedProgramState();
current.LastHashedProgramState = this.GetHashedProgramState();
}
// Try to enable any operations with satisfied dependencies before asking the
@ -880,7 +850,6 @@ namespace Microsoft.Coyote.Runtime
}
IO.Debug.WriteLine("<CoyoteDebug> Scheduling operation '{0}'.", next.Name);
this.ScheduleTrace.AddSchedulingChoice(next.Id);
if (current != next)
{
// Pause the currently scheduled operation, and enable the next one.
@ -934,7 +903,7 @@ namespace Microsoft.Coyote.Runtime
throw new ThreadInterruptedException();
}
current = this.GetExecutingOperation<ControlledOperation>();
current = this.GetExecutingOperation();
if (current != null)
{
// Choose the next delay to inject. The value is in milliseconds.
@ -968,7 +937,7 @@ namespace Microsoft.Coyote.Runtime
lock (this.SyncObject)
{
// Checks if the current operation is controlled by the runtime.
this.GetExecutingOperation<ControlledOperation>();
this.GetExecutingOperation();
// Checks if the scheduling steps bound has been reached.
this.CheckIfSchedulingStepsBoundIsReached();
@ -976,7 +945,7 @@ namespace Microsoft.Coyote.Runtime
if (this.Configuration.IsProgramStateHashingEnabled)
{
// Update the current operation with the hashed program state.
this.ScheduledOperation.HashedProgramState = this.GetHashedProgramState();
this.ScheduledOperation.LastHashedProgramState = this.GetHashedProgramState();
}
if (!this.Scheduler.GetNextBooleanChoice(this.ScheduledOperation, maxValue, out bool choice))
@ -984,7 +953,6 @@ namespace Microsoft.Coyote.Runtime
this.Detach(SchedulerDetachmentReason.BoundReached);
}
this.ScheduleTrace.AddNondeterministicBooleanChoice(choice);
return choice;
}
}
@ -1003,7 +971,7 @@ namespace Microsoft.Coyote.Runtime
lock (this.SyncObject)
{
// Checks if the current operation is controlled by the runtime.
this.GetExecutingOperation<ControlledOperation>();
this.GetExecutingOperation();
// Checks if the scheduling steps bound has been reached.
this.CheckIfSchedulingStepsBoundIsReached();
@ -1011,7 +979,7 @@ namespace Microsoft.Coyote.Runtime
if (this.Configuration.IsProgramStateHashingEnabled)
{
// Update the current operation with the hashed program state.
this.ScheduledOperation.HashedProgramState = this.GetHashedProgramState();
this.ScheduledOperation.LastHashedProgramState = this.GetHashedProgramState();
}
if (!this.Scheduler.GetNextIntegerChoice(this.ScheduledOperation, maxValue, out int choice))
@ -1019,7 +987,6 @@ namespace Microsoft.Coyote.Runtime
this.Detach(SchedulerDetachmentReason.BoundReached);
}
this.ScheduleTrace.AddNondeterministicIntegerChoice(choice);
return choice;
}
}
@ -1162,7 +1129,7 @@ namespace Microsoft.Coyote.Runtime
{
IO.Debug.WriteLine("<CoyoteDebug> Trying to enable any operation with satisfied dependencies.");
int retries = 0;
int attempt = 0;
uint enabledOpsCount = 0;
uint accumulatedDelay = 0;
while (true)
@ -1215,7 +1182,7 @@ namespace Microsoft.Coyote.Runtime
if (this.Configuration.IsPartiallyControlledConcurrencyEnabled)
{
// Compute the delta of enabled operations from the previous attempt.
uint enabledOpsDelta = retries is 0 ? 0 : enabledOpsCount - previousEnabledOpsCount;
uint enabledOpsDelta = attempt is 0 ? 0 : enabledOpsCount - previousEnabledOpsCount;
// This value is true if the current operation just completed and has uncontrolled source.
bool isSourceUnresolved = current.Status is OperationStatus.Completed && current.IsSourceUncontrolled;
@ -1233,17 +1200,20 @@ namespace Microsoft.Coyote.Runtime
bool isConcurrencyUnresolved = enabledOpsDelta is 0 && statusChanges is 0 &&
(isNoEnabledOpsCaseResolved || isSomeEnabledOpsCaseResolved);
// Retry if there is unresolved concurrency and any retries left, or if there are no enabled
// Retry if there is unresolved concurrency and any attempts left, or if there are no enabled
// operations and the accumulated delay is less than the specified deadlock timeout limit.
if ((++retries < 5 && isConcurrencyUnresolved) ||
if ((++attempt < 5 && isConcurrencyUnresolved) ||
(enabledOpsCount is 0 && accumulatedDelay < this.Configuration.DeadlockTimeout))
{
// Implement a simple retry logic to try resolve uncontrolled concurrency.
IO.Debug.WriteLine(
"<CoyoteDebug> Pausing operation '{0}' on thread '{1}' to try resolve uncontrolled concurrency.",
this.ScheduledOperation.Name, Thread.CurrentThread.ManagedThreadId);
"<CoyoteDebug> Pausing controlled thread '{0}' to try resolve uncontrolled concurrency.",
Thread.CurrentThread.ManagedThreadId);
uint delay = this.Configuration.UncontrolledConcurrencyTimeout;
// Necessary until we have a better synchronization mechanism to give
// more chance to another thread to resolve uncontrolled concurrency.
SyncMonitor.Wait(this.SyncObject, (int)delay);
Thread.Yield();
accumulatedDelay += delay;
continue;
}
@ -1322,17 +1292,20 @@ namespace Microsoft.Coyote.Runtime
{
// A scheduling point from an uncontrolled thread has not been postponed yet, so pause the execution
// of the current operation to try give time to the uncontrolled concurrency to be resolved.
if (!this.IsLastSchedulingPointPostponed)
if (this.LastPostponedSchedulingPoint is null)
{
IO.Debug.WriteLine(
"<CoyoteDebug> Pausing operation '{0}' on thread '{1}' to try resolve uncontrolled concurrency.",
op.Name, Thread.CurrentThread.ManagedThreadId);
int retries = 0;
"<CoyoteDebug> Pausing controlled thread '{0}' to try resolve uncontrolled concurrency.",
Thread.CurrentThread.ManagedThreadId);
int attempt = 0;
int delay = (int)this.Configuration.UncontrolledConcurrencyTimeout;
while (retries++ < 10 && !task.IsCompleted)
while (attempt++ < 10 && !task.IsCompleted)
{
// Necessary until we have a better synchronization mechanism to give
// more chance to another thread to resolve uncontrolled concurrency.
SyncMonitor.Wait(this.SyncObject, delay);
if (this.IsLastSchedulingPointPostponed)
Thread.Yield();
if (this.LastPostponedSchedulingPoint.HasValue)
{
// A scheduling point from an uncontrolled thread has been postponed,
// so stop trying to resolve the uncontrolled concurrency.
@ -1341,22 +1314,43 @@ namespace Microsoft.Coyote.Runtime
}
}
if (this.IsLastSchedulingPointPostponed)
if (this.LastPostponedSchedulingPoint.HasValue)
{
IO.Debug.WriteLine(
"<CoyoteDebug> Resuming operation '{0}' on thread '{1}' with uncontrolled concurrency resolved.",
op.Name, Thread.CurrentThread.ManagedThreadId);
this.ScheduleNextOperation(SchedulingPointType.Default, isSuppressible: false);
"<CoyoteDebug> Resuming controlled thread '{0}' with uncontrolled concurrency resolved.",
Thread.CurrentThread.ManagedThreadId);
this.ScheduleNextOperation(this.LastPostponedSchedulingPoint.Value, isSuppressible: false);
}
}
}
/// <summary>
/// Gets the <see cref="ControlledOperation"/> that is currently executing,
/// Returns the currently executing <see cref="ControlledOperation"/>,
/// or null if no such operation is executing.
/// </summary>
#if !DEBUG
[DebuggerStepThrough]
#endif
internal ControlledOperation GetExecutingOperation()
{
lock (this.SyncObject)
{
var op = ExecutingOperation.Value;
if (op is null)
{
this.NotifyUncontrolledCurrentThread();
}
return op;
}
}
/// <summary>
/// Returns the currently executing <see cref="ControlledOperation"/> of the
/// specified type, or null if no such operation is executing.
/// </summary>
#if !DEBUG
[DebuggerStepThrough]
#endif
internal TControlledOperation GetExecutingOperation<TControlledOperation>()
where TControlledOperation : ControlledOperation
@ -1374,18 +1368,31 @@ namespace Microsoft.Coyote.Runtime
}
/// <summary>
/// Gets the <see cref="ControlledOperation"/> associated with the specified
/// unique id, or null if no such operation exists.
/// Returns the <see cref="ControlledOperation"/> associated with the specified
/// operation id, or null if no such operation exists.
/// </summary>
internal ControlledOperation GetOperationWithId(ulong operationId)
{
lock (this.SyncObject)
{
this.OperationMap.TryGetValue(operationId, out ControlledOperation op);
return op;
}
}
/// <summary>
/// Returns the <see cref="ControlledOperation"/> of the specified type that is associated
/// with the specified operation id, or null if no such operation exists.
/// </summary>
#if !DEBUG
[DebuggerStepThrough]
#endif
internal TControlledOperation GetOperationWithId<TControlledOperation>(ulong id)
internal TControlledOperation GetOperationWithId<TControlledOperation>(ulong operationId)
where TControlledOperation : ControlledOperation
{
lock (this.SyncObject)
{
if (this.OperationMap.TryGetValue(id, out ControlledOperation op) &&
if (this.OperationMap.TryGetValue(operationId, out ControlledOperation op) &&
op is TControlledOperation expected)
{
return expected;
@ -1435,12 +1442,12 @@ namespace Microsoft.Coyote.Runtime
if (operation is ActorOperation actorOperation)
{
int operationHash = 31 + actorOperation.Actor.GetHashedState(this.SchedulingPolicy);
operationHash = (operationHash * 31) + actorOperation.SchedulingPoint.GetHashCode();
operationHash = (operationHash * 31) + actorOperation.LastSchedulingPoint.GetHashCode();
hash *= operationHash;
}
else
{
hash *= 31 + operation.SchedulingPoint.GetHashCode();
hash *= 31 + operation.LastSchedulingPoint.GetHashCode();
}
}
@ -1521,12 +1528,12 @@ namespace Microsoft.Coyote.Runtime
internal void MonitorTaskCompletion(Task task) => this.SpecificationEngine.MonitorTaskCompletion(task);
/// <summary>
/// Returns true if the specified thread is uncontrolled, else false.
/// Returns true if the specified thread is controlled, else false.
/// </summary>
internal bool IsThreadUncontrolled(Thread thread)
private bool IsThreadControlled(Thread thread)
{
string name = thread?.Name;
return name is null || !this.ControlledThreads.ContainsKey(name);
return name != null && this.ControlledThreads.ContainsKey(name);
}
/// <summary>
@ -1979,15 +1986,29 @@ namespace Microsoft.Coyote.Runtime
}
/// <summary>
/// Sets this runtime instance to the execution context of the current thread,
/// allowing future retrieval in the same thread, as well as across threads in
/// Sets up the context of the executing controlled thread, allowing future retrieval
/// of runtime related data from the same thread, as well as across threads that share
/// the same asynchronous control flow.
/// </summary>
private void SetThreadExecutionContext()
private void SetCurrentExecutionContext(ControlledOperation op)
{
ThreadLocalRuntime.Value = this;
AsyncLocalRuntime.Value = this;
this.SetControlledSynchronizationContext();
// Assign the specified controlled operation to the executing thread, and
// associate the operation group, if any, with the current context.
ExecutingOperation.Value = op;
}
/// <summary>
/// Removes any runtime related data from the context of the executing controlled thread.
/// </summary>
private static void CleanCurrentExecutionContext()
{
ExecutingOperation.Value = null;
AsyncLocalRuntime.Value = null;
ThreadLocalRuntime.Value = null;
}
/// <summary>
@ -2083,7 +2104,6 @@ namespace Microsoft.Coyote.Runtime
this.ControlledTaskScheduler.Dispose();
this.SyncContext.Dispose();
this.SpecificationEngine.Dispose();
this.ScheduleTrace.Dispose();
if (this.SchedulingPolicy is SchedulingPolicy.Systematic)
{

Просмотреть файл

@ -9,7 +9,7 @@ namespace Microsoft.Coyote.Runtime
/// <summary>
/// Represents an operation that can be controlled during testing.
/// </summary>
internal class ControlledOperation : IEquatable<ControlledOperation>
internal class ControlledOperation : IEquatable<ControlledOperation>, IDisposable
{
/// <summary>
/// The unique id of the operation.
@ -27,16 +27,21 @@ namespace Microsoft.Coyote.Runtime
/// </summary>
internal OperationStatus Status;
/// <summary>
/// The type of the last encountered scheduling point.
/// </summary>
internal SchedulingPointType SchedulingPoint;
/// <summary>
/// Set of dependencies that must get satisfied before this operation can resume executing.
/// </summary>
internal readonly HashSet<object> Dependencies;
/// <summary>
/// The type of the last encountered scheduling point.
/// </summary>
internal SchedulingPointType LastSchedulingPoint;
/// <summary>
/// A value that represents the hashed program state when this operation last executed.
/// </summary>
internal int LastHashedProgramState;
/// <summary>
/// True if the source of this operation is uncontrolled, else false.
/// </summary>
@ -47,12 +52,6 @@ namespace Microsoft.Coyote.Runtime
/// </summary>
internal bool IsAnyDependencyUncontrolled;
/// <summary>
/// A value that represents the hashed program state when
/// this operation last executed.
/// </summary>
internal int HashedProgramState;
/// <summary>
/// Initializes a new instance of the <see cref="ControlledOperation"/> class.
/// </summary>
@ -61,8 +60,9 @@ namespace Microsoft.Coyote.Runtime
this.Id = operationId;
this.Name = name;
this.Status = OperationStatus.None;
this.SchedulingPoint = SchedulingPointType.Start;
this.Dependencies = new HashSet<object>();
this.LastSchedulingPoint = SchedulingPointType.Start;
this.LastHashedProgramState = 0;
this.IsSourceUncontrolled = false;
this.IsAnyDependencyUncontrolled = false;
}
@ -124,7 +124,7 @@ namespace Microsoft.Coyote.Runtime
public override int GetHashCode() => this.Id.GetHashCode();
/// <summary>
/// Returns a string that represents the current actor id.
/// Returns a string that represents the current operation id.
/// </summary>
public override string ToString() => this.Name;
@ -139,5 +139,13 @@ namespace Microsoft.Coyote.Runtime
/// to the current <see cref="ControlledOperation"/>.
/// </summary>
bool IEquatable<ControlledOperation>.Equals(ControlledOperation other) => this.Equals(other);
/// <summary>
/// Disposes the contents of this object.
/// </summary>
public void Dispose()
{
this.Dependencies.Clear();
}
}
}

Просмотреть файл

@ -34,10 +34,10 @@ namespace Microsoft.Coyote.Runtime
/// </summary>
internal void Wait()
{
var op = this.Runtime.GetExecutingOperation<ControlledOperation>();
var op = this.Runtime.GetExecutingOperation();
op.Status = OperationStatus.BlockedOnResource;
this.AwaitingOperations.Add(op);
this.Runtime.ScheduleNextOperation(SchedulingPointType.Join);
this.Runtime.ScheduleNextOperation(SchedulingPointType.Wait);
}
/// <summary>

Просмотреть файл

@ -41,7 +41,7 @@ namespace Microsoft.Coyote.Runtime
{
IO.Debug.WriteLine("<CoyoteDebug> Enqueuing task '{0}' from thread '{1}'.",
task.Id, Thread.CurrentThread.ManagedThreadId);
this.Runtime.ScheduleTask(task);
this.Runtime.Schedule(task);
}
/// <inheritdoc/>

Просмотреть файл

@ -39,6 +39,11 @@ namespace Microsoft.Coyote.Runtime
/// </summary>
internal SchedulingPolicy SchedulingPolicy { get; private set; }
/// <summary>
/// The explored schedule trace.
/// </summary>
internal ScheduleTrace Trace { get; private set; }
/// <summary>
/// The count of exploration steps in the current iteration.
/// </summary>
@ -127,7 +132,12 @@ namespace Microsoft.Coyote.Runtime
/// </summary>
/// <param name="iteration">The id of the next iteration.</param>
/// <returns>True to start the specified iteration, else false to stop exploring.</returns>
internal bool InitializeNextIteration(uint iteration) => this.Strategy.InitializeNextIteration(iteration);
internal bool InitializeNextIteration(uint iteration)
{
this.Trace?.Dispose();
this.Trace = new ScheduleTrace();
return this.Strategy.InitializeNextIteration(iteration);
}
/// <summary>
/// Returns the next controlled operation to schedule.
@ -138,8 +148,18 @@ namespace Microsoft.Coyote.Runtime
/// <param name="next">The next operation to schedule.</param>
/// <returns>True if there is a next choice, else false.</returns>
internal bool GetNextOperation(IEnumerable<ControlledOperation> ops, ControlledOperation current,
bool isYielding, out ControlledOperation next) =>
(this.Strategy as SystematicStrategy).GetNextOperation(ops, current, isYielding, out next);
bool isYielding, out ControlledOperation next)
{
if (this.Strategy is SystematicStrategy systematicStrategy &&
systematicStrategy.GetNextOperation(ops, current, isYielding, out next))
{
this.Trace.AddSchedulingChoice(next.Id);
return true;
}
next = null;
return false;
}
/// <summary>
/// Returns the next boolean choice.
@ -148,8 +168,18 @@ namespace Microsoft.Coyote.Runtime
/// <param name="maxValue">The max value.</param>
/// <param name="next">The next boolean choice.</param>
/// <returns>True if there is a next choice, else false.</returns>
internal bool GetNextBooleanChoice(ControlledOperation current, int maxValue, out bool next) =>
(this.Strategy as SystematicStrategy).GetNextBooleanChoice(current, maxValue, out next);
internal bool GetNextBooleanChoice(ControlledOperation current, int maxValue, out bool next)
{
if (this.Strategy is SystematicStrategy systematicStrategy &&
systematicStrategy.GetNextBooleanChoice(current, maxValue, out next))
{
this.Trace.AddNondeterministicBooleanChoice(next);
return true;
}
next = false;
return false;
}
/// <summary>
/// Returns the next integer choice.
@ -158,8 +188,18 @@ namespace Microsoft.Coyote.Runtime
/// <param name="maxValue">The max value.</param>
/// <param name="next">The next integer choice.</param>
/// <returns>True if there is a next choice, else false.</returns>
internal bool GetNextIntegerChoice(ControlledOperation current, int maxValue, out int next) =>
(this.Strategy as SystematicStrategy).GetNextIntegerChoice(current, maxValue, out next);
internal bool GetNextIntegerChoice(ControlledOperation current, int maxValue, out int next)
{
if (this.Strategy is SystematicStrategy systematicStrategy &&
systematicStrategy.GetNextIntegerChoice(current, maxValue, out next))
{
this.Trace.AddNondeterministicIntegerChoice(next);
return true;
}
next = 0;
return false;
}
/// <summary>
/// Returns the next delay.

Просмотреть файл

@ -20,7 +20,7 @@ namespace Microsoft.Coyote.Runtime
var runtime = CoyoteRuntime.Current;
if (runtime.SchedulingPolicy is SchedulingPolicy.Systematic)
{
runtime.ScheduleNextOperation(SchedulingPointType.Default, isSuppressible: false);
runtime.ScheduleNextOperation(SchedulingPointType.Interleave, isSuppressible: false);
}
}

Просмотреть файл

@ -9,62 +9,82 @@ namespace Microsoft.Coyote.Runtime
internal enum SchedulingPointType
{
/// <summary>
/// The default scheduling point type.
/// The default scheduling point.
/// </summary>
Default = 0,
/// <summary>
/// A controlled operation starts executing.
/// </summary>
Start,
/// <summary>
/// A controlled operation was created.
/// </summary>
Create,
/// <summary>
/// A controlled operation sends an event.
/// A controlled operation started executing.
/// </summary>
Send,
Start,
/// <summary>
/// A controlled operation receives an event.
/// A controlled operation scheduled its continuation.
/// </summary>
Receive,
ContinueWith,
/// <summary>
/// A controlled operation halts executing.
/// A controlled operation yielded its execution.
/// </summary>
Halt,
Interleave,
/// <summary>
/// A controlled operation stops executing.
/// </summary>
Stop,
/// <summary>
/// A controlled operation yields its execution.
/// A controlled operation lowered its priority and yielded its execution.
/// </summary>
Yield,
/// <summary>
/// A controlled operation acquires a synchronized resource.
/// A controlled operation is reading shared state.
/// </summary>
Read,
/// <summary>
/// A controlled operation is writing shared state.
/// </summary>
Write,
/// <summary>
/// A controlled operation is blocked until one or more dependencies are satisfied.
/// </summary>
Wait,
/// <summary>
/// A controlled operation acquired a synchronized resource.
/// </summary>
Acquire,
/// <summary>
/// A controlled operation releases a synchronized resource.
/// A controlled operation released a synchronized resource.
/// </summary>
Release,
/// <summary>
/// A controlled operation waits for another operation to stop.
/// A controlled operation sent an event.
/// </summary>
Join,
Send,
/// <summary>
/// A controlled operation injects a failure.
/// A controlled operation received an event.
/// </summary>
Receive,
/// <summary>
/// A controlled operation completed its execution.
/// </summary>
Complete,
/// <summary>
/// A controlled operation halted executing.
/// </summary>
Halt,
/// <summary>
/// A controlled operation injected a failure.
/// </summary>
InjectFailure
}

Просмотреть файл

@ -1,7 +1,7 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
namespace Microsoft.Coyote.Testing
namespace Microsoft.Coyote.Runtime
{
/// <summary>
/// Class implementing a program schedule step.

Просмотреть файл

@ -1,7 +1,7 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
namespace Microsoft.Coyote.Testing
namespace Microsoft.Coyote.Runtime
{
/// <summary>
/// The schedule step type.

Просмотреть файл

@ -7,7 +7,7 @@ using System.Collections.Generic;
using System.IO;
using System.Text;
namespace Microsoft.Coyote.Testing
namespace Microsoft.Coyote.Runtime
{
/// <summary>
/// Class implementing a program schedule trace. A trace is a series

Просмотреть файл

@ -11,6 +11,10 @@ namespace Microsoft.Coyote.Testing.Systematic
/// <summary>
/// A probabilistic scheduling strategy that uses Q-learning.
/// </summary>
/// <remarks>
/// This strategy is described in the following paper:
/// https://dl.acm.org/doi/10.1145/3428298.
/// </remarks>
internal sealed class QLearningStrategy : RandomStrategy
{
/// <summary>
@ -260,10 +264,10 @@ namespace Microsoft.Coyote.Testing.Systematic
/// </summary>
private int CaptureExecutionStep(ControlledOperation current)
{
int state = current.HashedProgramState;
int state = current.LastHashedProgramState;
// Update the execution path with the current state.
this.ExecutionPath.AddLast((this.LastOperation, current.SchedulingPoint, state));
this.ExecutionPath.AddLast((this.LastOperation, current.LastSchedulingPoint, state));
if (!this.TransitionFrequencies.ContainsKey(state))
{

Просмотреть файл

@ -431,7 +431,7 @@ namespace Microsoft.Coyote.Rewriting.Types.Threading
{
if (this.Owner != null)
{
var op = this.Resource.Runtime.GetExecutingOperation<ControlledOperation>();
var op = this.Resource.Runtime.GetExecutingOperation();
return this.Owner == op;
}
@ -452,7 +452,7 @@ namespace Microsoft.Coyote.Rewriting.Types.Threading
if (this.Owner != null)
{
var op = this.Resource.Runtime.GetExecutingOperation<ControlledOperation>();
var op = this.Resource.Runtime.GetExecutingOperation();
if (this.Owner == op)
{
// The owner is re-entering the lock.
@ -476,7 +476,7 @@ namespace Microsoft.Coyote.Rewriting.Types.Threading
}
// The executing op acquired the lock and can proceed.
this.Owner = this.Resource.Runtime.GetExecutingOperation<ControlledOperation>();
this.Owner = this.Resource.Runtime.GetExecutingOperation();
this.LockCountMap.Add(this.Owner, 1);
return this;
}
@ -499,7 +499,7 @@ namespace Microsoft.Coyote.Rewriting.Types.Threading
/// </summary>
private void SchedulePulse(PulseOperation pulseOperation)
{
var op = this.Resource.Runtime.GetExecutingOperation<ControlledOperation>();
var op = this.Resource.Runtime.GetExecutingOperation();
if (this.Owner != op)
{
throw new SystemSynchronizationLockException();
@ -573,7 +573,7 @@ namespace Microsoft.Coyote.Rewriting.Types.Threading
/// </summary>
internal bool Wait()
{
var op = this.Resource.Runtime.GetExecutingOperation<ControlledOperation>();
var op = this.Resource.Runtime.GetExecutingOperation();
if (this.Owner != op)
{
throw new SystemSynchronizationLockException();
@ -649,7 +649,7 @@ namespace Microsoft.Coyote.Rewriting.Types.Threading
internal void Exit()
{
var op = this.Resource.Runtime.GetExecutingOperation<ControlledOperation>();
var op = this.Resource.Runtime.GetExecutingOperation();
this.Resource.Runtime.Assert(this.LockCountMap.ContainsKey(op), "Cannot invoke Dispose without acquiring the lock.");
this.LockCountMap[op]--;

Просмотреть файл

@ -219,7 +219,7 @@ namespace Microsoft.Coyote.Rewriting.Types.Threading
public override bool Wait(int millisecondsTimeout, SystemCancellationToken cancellationToken)
{
// TODO: support cancellations during testing.
this.Resource.Runtime.ScheduleNextOperation(SchedulingPointType.Join);
this.Resource.Runtime.ScheduleNextOperation(SchedulingPointType.Wait);
// We need this loop, because when a resource gets released it notifies all asynchronous
// operations waiting to acquire it, even if such an operation is still blocked.

Просмотреть файл

@ -345,7 +345,7 @@ namespace Microsoft.Coyote.SystematicTesting
else
{
Error.ReportAndExit("Exception thrown during testing outside the context of an actor, " +
"possibly in a test method. Please use /debug /v:2 to print more information.");
"possibly in a test method. Please enable debug verbosity to print more information.");
}
}
catch (Exception ex)
@ -500,7 +500,7 @@ namespace Microsoft.Coyote.SystematicTesting
runtime = new CoyoteRuntime(this.Configuration, this.Scheduler);
// If verbosity is turned off, then intercept the program log, and also redirect
// the standard output and error streams to a nul logger.
// the standard output and error streams to the runtime logger.
if (!this.Configuration.IsVerbose)
{
runtimeLogger = new InMemoryLogger();
@ -511,9 +511,8 @@ namespace Microsoft.Coyote.SystematicTesting
runtime.Logger = runtimeLogger;
var writer = TextWriter.Null;
Console.SetOut(writer);
Console.SetError(writer);
Console.SetOut(runtimeLogger.TextWriter);
Console.SetError(runtimeLogger.TextWriter);
}
else if (this.Logger != this.DefaultLogger)
{
@ -535,11 +534,6 @@ namespace Microsoft.Coyote.SystematicTesting
callback(iteration);
}
if (runtime.IsBugFound)
{
this.Logger.WriteLine(LogSeverity.Error, runtime.BugReport);
}
runtime.LogWriter.LogCompletion();
this.GatherTestingStatistics(runtime);
@ -560,7 +554,7 @@ namespace Microsoft.Coyote.SystematicTesting
if (runtime.SchedulingPolicy is SchedulingPolicy.Systematic)
{
this.ReproducibleTrace = runtime.ScheduleTrace.Serialize(
this.ReproducibleTrace = this.Scheduler.Trace.Serialize(
this.Configuration, this.Scheduler.IsScheduleFair);
}
}
@ -583,13 +577,16 @@ namespace Microsoft.Coyote.SystematicTesting
$"switching to fuzzing due to uncontrolled concurrency " +
$"[task-{this.Configuration.TestingProcessId}]");
}
if (runtime.IsBugFound && !this.Scheduler.IsReplayingSchedule &&
this.Configuration.RunTestIterationsToCompletion)
else if (runtime.IsBugFound)
{
this.Logger.WriteLine(LogSeverity.Important, $"..... Iteration #{iteration + 1} " +
$"triggered bug #{this.TestReport.NumOfFoundBugs} " +
$"[task-{this.Configuration.TestingProcessId}]");
if (!this.Scheduler.IsReplayingSchedule &&
this.Configuration.RunTestIterationsToCompletion)
{
this.Logger.WriteLine(LogSeverity.Important, $"..... Iteration #{iteration + 1} " +
$"triggered bug #{this.TestReport.NumOfFoundBugs} " +
$"[task-{this.Configuration.TestingProcessId}]");
}
this.Logger.WriteLine(LogSeverity.Error, runtime.BugReport);
}

Просмотреть файл

@ -1,5 +1,5 @@
{
"sdk": {
"version": "6.0.101"
"version": "6.0.200"
}
}