Implement CacheSizeTracker and use FASTER v2 (#128)

* update to FASTER v2 and implement MemoryTracker

* update history size calculation in tests

* fix renamed variable

* work around incorrect status on read completion callback

* minor updates to tests

* adjust timeout for latest bits

* Recover fewer pages.

* handle one more transient timeout on storage calls.

* fix compaction

(cherry picked from commit acec2f0ac06c25e39bda3edcb4ab6a7a8b71d5bb)

* fix client issues causing hangs

(cherry picked from commit 6434c56033ee9fac995659764ab2aedd75fb0d94)

* update to latest prerelease

(cherry picked from commit 0ad283aaa9956022d1e5bf14b616961a59f071d6)

* add EH overlap to work around bug

* add tolerance to MemoryReduction test

* overhaul exception handling in query events

* remove deadlock risks in cancellation token handlers

* compaction must be categorized like scan

* load fewer pages on recovery

* trace event id string when receiving packet

* add condition to size check to account for concurrent modifications

* keep termination continuation direct for tracing and for debugging

* retry slow reads to work around FASTER hang issues

* update tracing

* instrument hang detection

* update timeout limits

* refactor cancellation

* make the faultinjector back off of startup after 2 failures

* overhaul shutdown sequence

* add hang detection to azure storage device

* add CompactThenFail test

* simplify checkpoint state machine as compaction now already moves BeginAddress

* fix bug in PartitionErrorHandler

* Arm assertions in release build also, and run EH CI tests on release

* extend hang detection limit to 90s and remove the slowest ScaleSmallScenarios test

* update compaction progress reporting

* recognize yet another type of transient storage error

* use release build and remove concurrent test from EH tests on CI

* implement thread count watcher

* fix bug in AzureStorageDevice.RemoveSegmentsAsync and add to tracking

* remove .ConfigureAwait(false)

* update PerformanceTests http bindings to support arbitrary prefixes with start/purge/query/count/await

* fix CI

* fix nullref in dispose
This commit is contained in:
Sebastian Burckhardt 2022-03-09 16:40:18 -08:00 коммит произвёл GitHub
Родитель 301828b6ba
Коммит 177cf161e0
Не найден ключ, соответствующий данной подписи
Идентификатор ключа GPG: 4AEE18F83AFDEB23
74 изменённых файлов: 3654 добавлений и 1304 удалений

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

@ -12,7 +12,7 @@ pool:
variables:
solution: 'DurableTask.Netherite.sln'
buildPlatform: 'Any CPU'
buildConfiguration: 'Debug'
buildConfiguration: 'Release'
steps:
- task: NuGetToolInstaller@1

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

@ -117,7 +117,7 @@ namespace DurableTask.Netherite.AzureFunctions
if (this.Service.TryGetScalingMonitor(out var monitor))
{
scaleMonitor = new ScaleMonitor(monitor);
monitor.InformationTracer($"ScaleMonitor Constructed Microsoft.Azure.WebJobs.Host.Scale.IScaleMonitor {scaleMonitor.Descriptor}");
monitor.InformationTracer($"ScaleMonitor Constructed, Descriptor.Id={scaleMonitor.Descriptor.Id}");
return true;
}
else

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

@ -88,6 +88,9 @@ namespace DurableTask.Netherite.AzureFunctions
JsonConvert.PopulateObject(JsonConvert.SerializeObject(this.options), netheriteSettings);
JsonConvert.PopulateObject(JsonConvert.SerializeObject(this.options.StorageProvider), netheriteSettings);
// configure the cache size if not already configured
netheriteSettings.InstanceCacheSizeMB ??= (this.inConsumption ? 100 : 200 * Environment.ProcessorCount);
// if worker id is specified in environment, it overrides the configured setting
string workerId = Environment.GetEnvironmentVariable("WorkerId");
if (!string.IsNullOrEmpty(workerId))
@ -137,7 +140,13 @@ namespace DurableTask.Netherite.AzureFunctions
&& bool.TryParse(s, out bool x)
&& x);
if (attachFaultInjector || attachReplayChecker)
bool attachCacheDebugger =
(this.options.StorageProvider.TryGetValue("AttachCacheDebugger", out object val2)
&& val2 is string s2
&& bool.TryParse(s2, out bool x2)
&& x2);
if (attachFaultInjector || attachReplayChecker || attachCacheDebugger)
{
netheriteSettings.TestHooks = new TestHooks();
@ -149,6 +158,10 @@ namespace DurableTask.Netherite.AzureFunctions
{
netheriteSettings.TestHooks.ReplayChecker = new Faster.ReplayChecker(netheriteSettings.TestHooks);
}
if (attachCacheDebugger)
{
netheriteSettings.TestHooks.CacheDebugger = new Faster.CacheDebugger(netheriteSettings.TestHooks);
}
}
if (this.TraceToConsole || this.TraceToBlob)

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

@ -78,6 +78,7 @@ namespace DurableTask.Netherite
try
{
this.currentUpdate.ApplyTo(trackedObject, this);
trackedObject.Version++;
}
catch (Exception exception) when (!Utils.IsFatal(exception))
{
@ -196,9 +197,10 @@ namespace DurableTask.Netherite
break;
}
this.EventDetailTracer?.TraceEventProcessingStarted(commitLogPosition, readEvent, EventTraceHelper.EventCategory.ReadEvent, false);
if (isReady)
{
this.EventDetailTracer?.TraceEventProcessingStarted(commitLogPosition, readEvent, EventTraceHelper.EventCategory.ReadEvent, false);
readEvent.Fire(this.Partition);
}
}
@ -220,7 +222,7 @@ namespace DurableTask.Netherite
}
}
public async Task ProcessQueryResultAsync(PartitionQueryEvent queryEvent, IAsyncEnumerable<OrchestrationState> instances)
public async Task ProcessQueryResultAsync(PartitionQueryEvent queryEvent, IAsyncEnumerable<OrchestrationState> instances, Task exceptionTask = null)
{
(long commitLogPosition, long inputQueuePosition) = this.GetPositions();
this.Assert(!this.IsReplaying, "query events are never part of the replay");
@ -231,7 +233,7 @@ namespace DurableTask.Netherite
try
{
this.EventDetailTracer?.TraceEventProcessingStarted(commitLogPosition, queryEvent, EventTraceHelper.EventCategory.QueryEvent, false);
await queryEvent.OnQueryCompleteAsync(instances, this.Partition);
await queryEvent.OnQueryCompleteAsync(instances, exceptionTask, this.Partition);
}
catch (OperationCanceledException)
{

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

@ -7,6 +7,7 @@ namespace DurableTask.Netherite
using System.Collections.Generic;
using System.Text;
using System.Threading;
using System.Threading.Tasks;
/// <summary>
/// A handler for fatal or non-fatal errors encountered in a partition.
@ -18,6 +19,11 @@ namespace DurableTask.Netherite
/// </summary>
CancellationToken Token { get; }
/// <summary>
/// A place to subscribe (potentially non-instantaneous) cleanup actions that execute on a dedicated thread.
/// </summary>
event Action OnShutdown;
/// <summary>
/// A boolean indicating whether the partition is terminated.
/// </summary>
@ -28,6 +34,11 @@ namespace DurableTask.Netherite
/// </summary>
bool NormalTermination { get; }
/// <summary>
/// Wait for all termination operations to finish
/// </summary>
Task<bool> WaitForTermination(TimeSpan timeout);
/// <summary>
/// Error handling for the partition.
/// </summary>

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

@ -57,6 +57,11 @@ namespace DurableTask.Netherite
}
}
/// <summary>
/// An estimation of the size that this object takes up in memory
/// </summary>
public virtual long EstimatedSize => 0;
/// <summary>
/// Is called on all singleton objects once at the very beginning
/// </summary>

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

@ -19,6 +19,8 @@ namespace DurableTask.Netherite
public TrackedObjectKey(TrackedObjectType objectType) { this.ObjectType = objectType; this.InstanceId = null; }
public TrackedObjectKey(TrackedObjectType objectType, string instanceId) { this.ObjectType = objectType; this.InstanceId = instanceId; }
public int EstimatedSize => 16 + (this.InstanceId == null ? 0 : 4 + this.InstanceId.Length);
public enum TrackedObjectType
{
// singletons

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

@ -48,7 +48,7 @@
<PackageReference Include="Microsoft.Extensions.Logging.Abstractions" Version="3.1.5" />
<PackageReference Include="Microsoft.Azure.EventHubs.Processor" Version="4.3.2" />
<PackageReference Include="Microsoft.Azure.Storage.Blob" Version="11.2.3" />
<PackageReference Include="Microsoft.FASTER.Core" Version="1.9.6" />
<PackageReference Include="Microsoft.FASTER.Core" Version="2.0.0-preview.1.2" />
<PackageReference Include="Microsoft.Azure.DurableTask.Core" Version="2.7.0" />
<PackageReference Include="Newtonsoft.Json" Version="12.0.3" />
<PackageReference Include="Microsoft.SourceLink.GitHub" Version="1.1.1" PrivateAssets="All" />

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

@ -30,7 +30,7 @@ namespace DurableTask.Netherite
}
}
public abstract Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> result, Partition partition);
public abstract Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> result, Task exceptionTask, Partition partition);
public sealed override void DetermineEffects(EffectTracker effects)
{

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

@ -14,7 +14,7 @@ namespace DurableTask.Netherite
{
const int batchsize = 11;
public async override Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> instances, Partition partition)
public async override Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> instances, Task exceptionTask, Partition partition)
{
int totalcount = 0;

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

@ -24,7 +24,7 @@ namespace DurableTask.Netherite
trackedObject.Process(this, effects);
}
public async override Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> instances, Partition partition)
public async override Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> instances, Task exceptionTask, Partition partition)
{
int batchCount = 0;

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

@ -24,12 +24,6 @@ namespace DurableTask.Netherite
[IgnoreDataMember]
public override string TracedInstanceId => this.InstanceId;
protected override void ExtraTraceInformation(StringBuilder s)
{
s.Append(' ');
s.Append(this.InstanceId);
}
public override void OnReadComplete(TrackedObject target, Partition partition)
{
var orchestrationState = ((InstanceState)target)?.OrchestrationState;

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

@ -44,7 +44,7 @@ namespace DurableTask.Netherite
protected override void ExtraTraceInformation(StringBuilder s)
{
base.ExtraTraceInformation(s);
s.Append(' ');
s.Append(this.NumActivitiesToSend);
s.Append("->Part");
s.Append(this.TransferDestination.ToString("D2"));

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

@ -47,7 +47,6 @@ namespace DurableTask.Netherite
foreach (var instanceId in this.InstanceIds)
{
yield return TrackedObjectKey.Instance(instanceId);
yield return TrackedObjectKey.History(instanceId);
}
}
}

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

@ -17,7 +17,8 @@ namespace DurableTask.Netherite
/// The continuation for the query operation.
/// </summary>
/// <param name="result">The tracked objects returned by this query</param>
/// <param name="exceptionTask">A task that throws an exception if the enumeration fails</param>
/// <param name="partition">The partition</param>
public abstract Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> result, Partition partition);
public abstract Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> result, Task exceptionTask, Partition partition);
}
}

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

@ -30,6 +30,25 @@ namespace DurableTask.Netherite
/// <param name="partition">The partition</param>
public abstract void OnReadComplete(TrackedObject target, Partition partition);
protected override void ExtraTraceInformation(StringBuilder s)
{
int completed, total;
if (this.Prefetch.HasValue)
{
completed = (this.prefetchLoaded ? 1 : 0) + (this.targetLoaded ? 1 : 0);
total = 2;
}
else
{
completed = this.targetLoaded ? 1 : 0;
total = 1;
}
s.Append(' ');
s.Append(completed);
s.Append('/');
s.Append(total);
}
#region prefetch state machine
[IgnoreDataMember]

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

@ -205,7 +205,7 @@ namespace DurableTask.Netherite
this.partitionEventId = partitionEventId;
this.client = client;
this.timeoutKey = (due, timeoutId);
this.continuation = new TaskCompletionSource<ClientEvent>();
this.continuation = new TaskCompletionSource<ClientEvent>(TaskCreationOptions.RunContinuationsAsynchronously);
this.startTime = this.client.workItemStopwatch.Elapsed.TotalMilliseconds;
}

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

@ -31,6 +31,7 @@ namespace DurableTask.Netherite
readonly ITaskHub taskHub;
readonly TransportConnectionString.StorageChoices configuredStorage;
readonly TransportConnectionString.TransportChoices configuredTransport;
readonly MemoryTracker memoryTracker;
readonly WorkItemTraceHelper workItemTraceHelper;
@ -43,6 +44,7 @@ namespace DurableTask.Netherite
CancellationTokenSource serviceShutdownSource;
Exception startupException;
Timer threadWatcher;
internal async ValueTask<Client> GetClientAsync()
{
@ -123,6 +125,8 @@ namespace DurableTask.Netherite
{
throw new NotSupportedException("Netherite backend requires 64bit, but current process is 32bit.");
}
this.memoryTracker = new MemoryTracker((long) (settings.InstanceCacheSizeMB ?? 400) * 1024 * 1024);
}
switch (this.configuredTransport)
@ -199,6 +203,16 @@ namespace DurableTask.Netherite
return false;
}
public void WatchThreads(object _)
{
if (TrackedThreads.NumberThreads > 100)
{
this.TraceHelper.TraceError("Too many threads, shutting down", TrackedThreads.GetThreadNames());
Thread.Sleep(TimeSpan.FromSeconds(60));
System.Environment.Exit(333);
}
}
/******************************/
@ -213,13 +227,7 @@ namespace DurableTask.Netherite
return new MemoryStorage(this.TraceHelper.Logger);
case TransportConnectionString.StorageChoices.Faster:
return new Faster.FasterStorage(
this.Settings.ResolvedStorageConnectionString,
this.Settings.ResolvedPageBlobStorageConnectionString,
this.Settings.UseLocalDirectoryForPartitionStorage,
this.Settings.HubName,
this.PathPrefix,
this.LoggerFactory);
return new Faster.FasterStorage(this.Settings, this.PathPrefix, this.memoryTracker, this.LoggerFactory);
default:
throw new NotImplementedException("no such storage choice");
@ -361,6 +369,11 @@ namespace DurableTask.Netherite
{
this.TraceHelper.TraceProgress("Starting Client");
if (this.Settings.TestHooks != null)
{
this.TraceHelper.TraceProgress(this.Settings.TestHooks.ToString());
}
this.serviceShutdownSource = new CancellationTokenSource();
await this.taskHub.StartClientAsync();
@ -424,6 +437,11 @@ namespace DurableTask.Netherite
this.TraceHelper.TraceWarning($"Ignoring configuration setting partitionCount={this.Settings.PartitionCount} because existing TaskHub has {this.NumberPartitions} partitions");
}
if (this.threadWatcher == null)
{
this.threadWatcher = new Timer(this.WatchThreads, null, 0, 120000);
}
this.TraceHelper.TraceProgress($"Started partitionCount={this.NumberPartitions}");
return ServiceState.Full;
@ -473,6 +491,9 @@ namespace DurableTask.Netherite
this.OrchestrationWorkItemQueue.Dispose();
}
this.threadWatcher?.Dispose();
this.threadWatcher = null;
this.TraceHelper.TraceProgress("Stopped cleanly");
return ServiceState.None;

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

@ -90,6 +90,11 @@ namespace DurableTask.Netherite
/// </summary>
public int ActivityDispatcherCount { get; set; } = 1;
/// <summary>
/// Limit for how much memory on each node should be used for caching instance states and histories
/// </summary>
public int? InstanceCacheSizeMB { get; set; } = null;
/// <summary>
/// Gets or sets the partition management option
/// </summary>

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

@ -31,7 +31,8 @@ namespace DurableTask.Netherite
public IPartitionErrorHandler ErrorHandler { get; private set; }
public PartitionTraceHelper TraceHelper { get; private set; }
public WorkItemTraceHelper WorkItemTraceHelper { get; private set; }
public Faster.CacheDebugger CacheDebugger { get; private set; }
public Faster.MemoryTracker CacheTracker { get; private set; }
public double CurrentTimeMs => this.stopwatch.Elapsed.TotalMilliseconds;
@ -66,6 +67,7 @@ namespace DurableTask.Netherite
WorkItemQueue<ActivityWorkItem> activityWorkItemQueue,
WorkItemQueue<OrchestrationWorkItem> orchestrationWorkItemQueue,
LoadPublisher loadPublisher,
WorkItemTraceHelper workItemTraceHelper)
{
this.host = host;
@ -83,7 +85,6 @@ namespace DurableTask.Netherite
this.WorkItemTraceHelper = workItemTraceHelper;
this.stopwatch.Start();
this.LastTransition = this.CurrentTimeMs;
this.CacheDebugger = this.Settings.TestHooks?.CacheDebugger;
}
public async Task<long> CreateOrRestoreAsync(IPartitionErrorHandler errorHandler, long firstInputQueuePosition)
@ -94,14 +95,19 @@ namespace DurableTask.Netherite
this.TraceHelper.TracePartitionProgress("Starting", ref this.LastTransition, this.CurrentTimeMs, "");
errorHandler.Token.Register(() => {
errorHandler.Token.Register(() =>
{
this.TraceHelper.TracePartitionProgress("Terminated", ref this.LastTransition, this.CurrentTimeMs, "");
if (!this.ErrorHandler.NormalTermination && this.Settings.TestHooks != null && this.Settings.TestHooks.FaultInjector == null)
if (!this.ErrorHandler.NormalTermination
&& this.Settings.TestHooks != null
&& this.Settings.TestHooks.FaultInjectionActive != true)
{
this.Settings.TestHooks.Error("Partition", "Unexpected partition termination during test");
this.Settings.TestHooks.Error("Partition", $"Unexpected termination of partition {this.PartitionId} during test");
}
});
}, useSynchronizationContext: false);
await MaxConcurrentStarts.WaitAsync();
@ -142,7 +148,6 @@ namespace DurableTask.Netherite
}
}
[Conditional("DEBUG")]
public void Assert(bool condition, string message)
{
if (!condition)

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

@ -5,6 +5,8 @@ namespace DurableTask.Netherite
{
using System;
using System.Threading;
using System.Threading.Tasks;
using Microsoft.Azure.Documents.SystemFunctions;
using Microsoft.Extensions.Logging;
// For indicating and initiating termination, and for tracing errors and warnings relating to a partition.
@ -17,11 +19,33 @@ namespace DurableTask.Netherite
readonly LogLevel logLevelLimit;
readonly string account;
readonly string taskHub;
readonly TaskCompletionSource<object> shutdownComplete;
public CancellationToken Token => this.cts.Token;
public bool IsTerminated => this.cts.Token.IsCancellationRequested;
public event Action OnShutdown;
public bool NormalTermination { get; private set; }
public CancellationToken Token
{
get
{
try
{
return this.cts.Token;
}
catch (ObjectDisposedException)
{
return new CancellationToken(true);
}
}
}
public bool IsTerminated => this.terminationStatus != NotTerminated;
public bool NormalTermination => this.terminationStatus == TerminatedNormally;
volatile int terminationStatus = NotTerminated;
const int NotTerminated = 0;
const int TerminatedWithError = 1;
const int TerminatedNormally = 2;
public PartitionErrorHandler(int partitionId, ILogger logger, LogLevel logLevelLimit, string storageAccountName, string taskHubName)
{
@ -31,14 +55,27 @@ namespace DurableTask.Netherite
this.logLevelLimit = logLevelLimit;
this.account = storageAccountName;
this.taskHub = taskHubName;
this.shutdownComplete = new TaskCompletionSource<object>();
}
public void HandleError(string context, string message, Exception exception, bool terminatePartition, bool isWarning)
{
this.TraceError(isWarning, context, message, exception, terminatePartition);
// terminate this partition in response to the error
if (terminatePartition && !this.cts.IsCancellationRequested)
if (terminatePartition && this.terminationStatus == NotTerminated)
{
if (Interlocked.CompareExchange(ref this.terminationStatus, TerminatedWithError, NotTerminated) == NotTerminated)
{
this.Terminate();
}
}
}
public void TerminateNormally()
{
if (Interlocked.CompareExchange(ref this.terminationStatus, TerminatedNormally, NotTerminated) == NotTerminated)
{
this.Terminate();
}
@ -65,30 +102,67 @@ namespace DurableTask.Netherite
}
}
public void TerminateNormally()
{
this.NormalTermination = true;
this.Terminate();
}
void Terminate()
{
// we use a dedicated shutdown thread to help debugging and to contain damage if there are hangs
Thread shutdownThread = TrackedThreads.MakeTrackedThread(Shutdown, "PartitionShutdown");
try
{
this.logger?.LogDebug("Part{partition:D2} Started PartitionCancellation");
this.cts.Cancel();
this.logger?.LogDebug("Part{partition:D2} Completed PartitionCancellation");
}
catch (AggregateException aggregate)
{
foreach (var e in aggregate.InnerExceptions)
{
this.HandleError("PartitionErrorHandler.Terminate", "Encountered exeption while canceling token", e, false, true);
this.HandleError("PartitionErrorHandler.Terminate", "Exception in PartitionCancellation", e, false, true);
}
}
catch (Exception e)
{
this.HandleError("PartitionErrorHandler.Terminate", "Encountered exeption while canceling token", e, false, true);
this.HandleError("PartitionErrorHandler.Terminate", "Exception in PartitionCancellation", e, false, true);
}
shutdownThread.Start();
void Shutdown()
{
try
{
this.logger?.LogDebug("Part{partition:D2} Started PartitionShutdown");
if (this.OnShutdown != null)
{
this.OnShutdown();
}
this.cts.Dispose();
this.logger?.LogDebug("Part{partition:D2} Completed PartitionShutdown");
}
catch (AggregateException aggregate)
{
foreach (var e in aggregate.InnerExceptions)
{
this.HandleError("PartitionErrorHandler.Shutdown", "Exception in PartitionShutdown", e, false, true);
}
}
catch (Exception e)
{
this.HandleError("PartitionErrorHandler.Shutdown", "Exception in PartitionShutdown", e, false, true);
}
this.shutdownComplete.TrySetResult(null);
}
}
public async Task<bool> WaitForTermination(TimeSpan timeout)
{
Task timeoutTask = Task.Delay(timeout);
var first = await Task.WhenAny(timeoutTask, this.shutdownComplete.Task);
return first == this.shutdownComplete.Task;
}
}
}

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

@ -10,7 +10,7 @@ namespace DurableTask.Netherite
/// </summary>
public class TestHooks
{
internal Faster.CacheDebugger CacheDebugger { get; set; }
public Faster.CacheDebugger CacheDebugger { get; set; }
public Faster.ReplayChecker ReplayChecker { get; set; }
@ -19,6 +19,9 @@ namespace DurableTask.Netherite
public Faster.CheckpointInjector CheckpointInjector { get; set; }
internal event Action<string> OnError;
bool launchDebugger = false; // may set this to true when hunting down bugs locally
public bool FaultInjectionActive => this.FaultInjector != null || (this.CheckpointInjector?.InjectFaultAfterCompaction == true);
internal void Error(string source, string message)
{
@ -26,6 +29,11 @@ namespace DurableTask.Netherite
{
System.Diagnostics.Debugger.Break();
}
else if (this.launchDebugger)
{
this.launchDebugger = false; // don't launch another one if the user detaches
System.Diagnostics.Debugger.Launch();
}
if (this.OnError != null)
{
this.OnError($"TestHook-{source} !!! {message}");
@ -33,5 +41,10 @@ namespace DurableTask.Netherite
Console.Error.WriteLine($"TestHook-{source} !!! {message}");
System.Diagnostics.Trace.TraceError($"TestHook-{source} !!! {message}");
}
public override string ToString()
{
return $"TestHooks:{(this.CacheDebugger != null ? " CacheDebugger" : "")}{(this.ReplayChecker != null ? " ReplayChecker" : "")}{(this.FaultInjector != null ? " FaultInjector" : "")}";
}
}
}

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

@ -75,7 +75,7 @@ namespace DurableTask.Netherite
public override void Process(RecoveryCompleted evt, EffectTracker effects)
{
effects.Partition.Assert(this.Pending.Count == evt.NumActivities, "count does not match in ActivitiesState.Process(RecoveryCompleted)");
effects.Partition.Assert(this.Pending.Count == evt.NumActivities, "count does not match in ActivitiesState.Process(RecoveryCompleted), actual={this.Pending.Count}");
foreach (var kvp in this.Pending)
{
@ -137,7 +137,7 @@ namespace DurableTask.Netherite
public override string ToString()
{
return $"Activities ({this.Pending.Count} pending) next={this.SequenceNumber:D6}";
return $"Activities Pending.Count={this.Pending.Count} next={this.SequenceNumber:D6}";
}
void ScheduleNextOffloadDecision(TimeSpan delay)

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

@ -32,6 +32,9 @@ namespace DurableTask.Netherite
[DataMember]
public int Episode { get; set; }
[DataMember]
public long HistorySize;
/// <summary>
/// We cache this so we can resume the execution at the execution cursor.
/// </summary>
@ -43,9 +46,13 @@ namespace DurableTask.Netherite
public override string ToString()
{
return $"History InstanceId={this.InstanceId} ExecutionId={this.ExecutionId} Events={this.History.Count}";
return $"History InstanceId={this.InstanceId} ExecutionId={this.ExecutionId} Events={this.History.Count} Size={this.HistorySize}";
}
public override long EstimatedSize => 60
+ 2 * ((this.InstanceId?.Length ?? 0) + (this.ExecutionId?.Length ?? 0) + (this.CustomStatus?.Length ?? 0))
+ this.HistorySize;
public override void Process(BatchProcessed evt, EffectTracker effects)
{
// can add events to the history, or replace it with a new history
@ -57,6 +64,7 @@ namespace DurableTask.Netherite
this.CustomStatus = null;
this.Episode = 0;
this.ExecutionId = evt.ExecutionId;
this.HistorySize = 0;
}
this.Partition.Assert(!string.IsNullOrEmpty(this.InstanceId) || string.IsNullOrEmpty(this.ExecutionId), "null ids in HistoryState.Process(BatchProcessed)");
@ -71,7 +79,8 @@ namespace DurableTask.Netherite
{
this.Episode++;
}
this.History.Add(evt.NewEvents[i]);
this.History.Add(historyEvent);
this.HistorySize += 8 + SizeUtils.GetEstimatedSize(historyEvent);
}
}
@ -89,6 +98,7 @@ namespace DurableTask.Netherite
evt.OrchestrationStatus,
this.History.Count,
evt.NewEvents,
this.HistorySize,
this.Episode);
// if present, we keep the work item so we can reuse the execution cursor

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

@ -21,6 +21,9 @@ namespace DurableTask.Netherite
[DataMember]
public OrchestrationState OrchestrationState { get; set; }
[DataMember]
public long OrchestrationStateSize { get; set; }
[DataMember]
public List<WaitRequestReceived> Waiters { get; set; }
@ -32,6 +35,8 @@ namespace DurableTask.Netherite
return $"Instance InstanceId={this.InstanceId} Status={this.OrchestrationState?.OrchestrationStatus}";
}
public override long EstimatedSize => 60 + 2 * (this.InstanceId?.Length ?? 0) + this.OrchestrationStateSize;
public override void Process(CreationRequestReceived creationRequestReceived, EffectTracker effects)
{
bool exists = this.OrchestrationState != null;
@ -59,6 +64,7 @@ namespace DurableTask.Netherite
CompletedTime = Core.Common.DateTimeUtils.MinDateTime,
ScheduledStartTime = ee.ScheduledStartTime
};
this.OrchestrationStateSize = DurableTask.Netherite.SizeUtils.GetEstimatedSize(this.OrchestrationState);
// queue the message in the session, or start a timer if delayed
if (!ee.ScheduledStartTime.HasValue)
@ -115,6 +121,7 @@ namespace DurableTask.Netherite
}
this.OrchestrationState.LastUpdatedTime = evt.Timestamp;
this.OrchestrationStateSize = DurableTask.Netherite.SizeUtils.GetEstimatedSize(this.OrchestrationState);
// if the orchestration is complete, notify clients that are waiting for it
if (this.Waiters != null)
@ -223,8 +230,11 @@ namespace DurableTask.Netherite
public override void Process(PurgeBatchIssued purgeBatchIssued, EffectTracker effects)
{
OrchestrationState state = this.OrchestrationState;
if (this.OrchestrationState != null
&& purgeBatchIssued.InstanceQuery.Matches(this.OrchestrationState))
bool matchesQuery = (state != null) && purgeBatchIssued.InstanceQuery.Matches(state);
effects.EventDetailTracer?.TraceEventProcessingDetail($"status={state?.OrchestrationStatus} matchesQuery={matchesQuery}");
if (matchesQuery)
{
purgeBatchIssued.Purged.Add(this.InstanceId);

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

@ -92,6 +92,7 @@ namespace DurableTask.Netherite
{
s.Append(':');
s.Append(this.request.ToString());
base.ExtraTraceInformation(s);
}
public override TrackedObjectKey ReadTarget => this.request.Target;

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

@ -101,11 +101,11 @@ namespace DurableTask.Netherite
public override Netherite.InstanceQuery InstanceQuery => this.request.InstanceQuery;
public override async Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> result, Partition partition)
public override async Task OnQueryCompleteAsync(IAsyncEnumerable<OrchestrationState> result, Task exceptionTask, Partition partition)
{
partition.Assert(this.request.Phase == ClientRequestEventWithQuery.ProcessingPhase.Query, "wrong phase in QueriesState.OnQueryCompleteAsync");
await this.request.OnQueryCompleteAsync(result, partition);
await this.request.OnQueryCompleteAsync(result, exceptionTask, partition);
// we now how to recycle the request event again in order to remove it from the list of pending queries
var again = (ClientRequestEventWithQuery)this.request.Clone();

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

@ -108,7 +108,7 @@ namespace DurableTask.Netherite
public override string ToString()
{
return $"Sessions ({this.Sessions.Count} pending) next={this.SequenceNumber:D6}";
return $"Sessions Count={this.Sessions.Count} next={this.SequenceNumber:D6}";
}
string GetSessionId(Session session) => $"{this.Partition.PartitionId:D2}S{session.SessionId}";
@ -324,6 +324,8 @@ namespace DurableTask.Netherite
// a different session id
if (!this.Sessions.TryGetValue(evt.InstanceId, out var session) || session.SessionId != evt.SessionId)
{
this.Partition.EventDetailTracer?.TraceEventProcessingDetail($"discarded evtsession={evt.SessionId} actualsession={(session != null ? this.GetSessionPosition(session).ToString() : "none")}");
if (!effects.IsReplaying)
{
this.Partition.WorkItemTraceHelper.TraceWorkItemDiscarded(

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

@ -84,6 +84,8 @@ namespace DurableTask.Netherite.Scaling
var batch = await this.table.ExecuteQuerySegmentedAsync<PartitionInfoEntity>(query, continuationToken, null, null, cancellationToken).ConfigureAwait(false);
foreach (var e in batch)
{
int.TryParse(e.CachePct, out int cachePct);
double.TryParse(e.MissRate, out double missRatePct);
result.Add(e.PartitionId, new PartitionLoadInfo()
{
WorkItems = e.WorkItems,
@ -97,7 +99,9 @@ namespace DurableTask.Netherite.Scaling
CommitLogPosition = e.CommitLogPosition,
WorkerId = e.WorkerId,
LatencyTrend = e.LatencyTrend,
MissRate = e.MissRate,
MissRate = missRatePct / 100,
CachePct = cachePct,
CacheMB = e.CacheMB,
});
}
}
@ -118,7 +122,9 @@ namespace DurableTask.Netherite.Scaling
public long CommitLogPosition { get; set; }
public string WorkerId { get; set; }
public string LatencyTrend { get; set; }
public double MissRate { get; set; }
public string MissRate { get; set; }
public string CachePct { get; set; }
public double CacheMB { get; set; }
public PartitionInfoEntity()
{
@ -146,7 +152,9 @@ namespace DurableTask.Netherite.Scaling
this.CommitLogPosition = info.CommitLogPosition;
this.WorkerId = info.WorkerId;
this.LatencyTrend = info.LatencyTrend;
this.MissRate = info.MissRate;
this.MissRate = $"{info.MissRate*100:f2}%";
this.CachePct = $"{info.CachePct}%";
this.CacheMB = info.CacheMB;
this.ETag = "*"; // no conditions when inserting, replace existing
}

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

@ -121,6 +121,18 @@ namespace DurableTask.Netherite.Scaling
[DataMember]
public double MissRate { get; set; }
/// <summary>
/// The current size of the cache, as a percentage.
/// </summary>
[DataMember]
public int CachePct { get; set; }
/// <summary>
/// The current size of the cache in MB
/// </summary>
[DataMember]
public double CacheMB { get; set; }
/// <summary>
/// The character representing idle load.
/// </summary>

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

@ -8,6 +8,7 @@ namespace DurableTask.Netherite.Faster
using System.Diagnostics;
using System.IO;
using System.Linq;
using System.Threading;
using System.Threading.Tasks;
using DurableTask.Core.Common;
using FASTER.core;
@ -25,6 +26,28 @@ namespace DurableTask.Netherite.Faster
readonly CloudBlobDirectory pageBlobDirectory;
readonly string blobName;
readonly bool underLease;
readonly ConcurrentDictionary<long, ReadWriteRequestInfo> pendingReadWriteOperations;
readonly ConcurrentDictionary<long, RemoveRequestInfo> pendingRemoveOperations;
readonly Timer hangCheckTimer;
readonly TimeSpan limit;
static long sequenceNumber;
struct ReadWriteRequestInfo
{
public bool IsRead;
public DeviceIOCompletionCallback Callback;
public uint NumBytes;
public object Context;
public DateTime TimeStamp;
}
struct RemoveRequestInfo
{
public AsyncCallback Callback;
public IAsyncResult Result;
public DateTime TimeStamp;
}
internal IPartitionErrorHandler PartitionErrorHandler { get; private set; }
@ -50,12 +73,17 @@ namespace DurableTask.Netherite.Faster
: base($"{blockBlobDirectory}\\{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED)
{
this.blobs = new ConcurrentDictionary<int, BlobEntry>();
this.pendingReadWriteOperations = new ConcurrentDictionary<long, ReadWriteRequestInfo>();
this.pendingRemoveOperations = new ConcurrentDictionary<long, RemoveRequestInfo>();
this.blockBlobDirectory = blockBlobDirectory;
this.pageBlobDirectory = pageBlobDirectory;
this.blobName = blobName;
this.PartitionErrorHandler = blobManager.PartitionErrorHandler;
this.PartitionErrorHandler.Token.Register(this.CancelAllRequests);
this.BlobManager = blobManager;
this.underLease = underLease;
this.hangCheckTimer = new Timer(this.CheckForHangs, null, 0, 20000);
this.limit = TimeSpan.FromSeconds(60);
}
/// <inheritdoc/>
@ -66,86 +94,96 @@ namespace DurableTask.Netherite.Faster
public async Task StartAsync()
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"AzureStorageDevice.StartAsync Called target={this.pageBlobDirectory.Prefix}{this.blobName}");
// list all the blobs representing the segments
var prefix = $"{this.blockBlobDirectory.Prefix}{this.blobName}.";
BlobContinuationToken continuationToken = null;
do
try
{
if (this.underLease)
{
await this.BlobManager.ConfirmLeaseIsGoodForAWhileAsync().ConfigureAwait(false);
}
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.StartAsync target={this.pageBlobDirectory.Prefix}{this.blobName}");
BlobResultSegment response = null;
await this.BlobManager.PerformWithRetriesAsync(
BlobManager.AsynchronousStorageReadMaxConcurrency,
true,
"PageBlobDirectory.ListBlobsSegmentedAsync",
"RecoverDevice",
$"continuationToken={continuationToken}",
this.pageBlobDirectory.Prefix,
2000,
true,
async (numAttempts) => {
response = await this.pageBlobDirectory.ListBlobsSegmentedAsync(
useFlatBlobListing: false,
blobListingDetails: BlobListingDetails.None,
maxResults: 100,
currentToken: continuationToken,
options: BlobManager.BlobRequestOptionsWithRetry,
operationContext: null);
return response.Results.Count(); // not accurate, in terms of bytes, but still useful for tracing purposes
});
foreach (IListBlobItem item in response.Results)
// list all the blobs representing the segments
var prefix = $"{this.blockBlobDirectory.Prefix}{this.blobName}.";
BlobContinuationToken continuationToken = null;
do
{
if (item is CloudPageBlob pageBlob)
if (this.underLease)
{
if (Int32.TryParse(pageBlob.Name.Replace(prefix, ""), out int segmentId))
await this.BlobManager.ConfirmLeaseIsGoodForAWhileAsync();
}
BlobResultSegment response = null;
await this.BlobManager.PerformWithRetriesAsync(
BlobManager.AsynchronousStorageReadMaxConcurrency,
true,
"PageBlobDirectory.ListBlobsSegmentedAsync",
"RecoverDevice",
$"continuationToken={continuationToken}",
this.pageBlobDirectory.Prefix,
2000,
true,
async (numAttempts) =>
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"AzureStorageDevice.StartAsync found segment={pageBlob.Name}");
response = await this.pageBlobDirectory.ListBlobsSegmentedAsync(
useFlatBlobListing: false,
blobListingDetails: BlobListingDetails.None,
maxResults: 100,
currentToken: continuationToken,
options: BlobManager.BlobRequestOptionsWithRetry,
operationContext: null);
return response.Results.Count(); // not accurate, in terms of bytes, but still useful for tracing purposes
});
bool ret = this.blobs.TryAdd(segmentId, new BlobEntry(pageBlob, this));
if (!ret)
foreach (IListBlobItem item in response.Results)
{
if (item is CloudPageBlob pageBlob)
{
if (Int32.TryParse(pageBlob.Name.Replace(prefix, ""), out int segmentId))
{
throw new InvalidOperationException("Recovery of blobs is single-threaded and should not yield any failure due to concurrency");
this.BlobManager?.StorageTracer?.FasterStorageProgress($"AzureStorageDevice.StartAsync found segment={pageBlob.Name}");
bool ret = this.blobs.TryAdd(segmentId, new BlobEntry(pageBlob, this));
if (!ret)
{
throw new InvalidOperationException("Recovery of blobs is single-threaded and should not yield any failure due to concurrency");
}
}
}
}
continuationToken = response.ContinuationToken;
}
continuationToken = response.ContinuationToken;
}
while (continuationToken != null);
while (continuationToken != null);
// find longest contiguous sequence at end
var keys = this.blobs.Keys.ToList();
if (keys.Count == 0)
{
// nothing has been written to this device so far.
this.startSegment = 0;
this.endSegment = -1;
}
else
{
keys.Sort();
this.endSegment = keys.Last();
for (int i = keys.Count - 2; i >= 0; i--)
// find longest contiguous sequence at end
var keys = this.blobs.Keys.ToList();
if (keys.Count == 0)
{
if (keys[i] == keys[i + 1] - 1)
// nothing has been written to this device so far.
this.startSegment = 0;
this.endSegment = -1;
}
else
{
keys.Sort();
this.endSegment = keys.Last();
for (int i = keys.Count - 2; i >= 0; i--)
{
this.startSegment = i;
if (keys[i] == keys[i + 1] - 1)
{
this.startSegment = i;
}
}
}
}
this.BlobManager?.StorageTracer?.FasterStorageProgress($"AzureStorageDevice.StartAsync Returned, determined segment range for {this.pageBlobDirectory.Prefix}{this.blobName}: start={this.startSegment} end={this.endSegment}");
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.StartAsync, determined segment range for {this.pageBlobDirectory.Prefix}{this.blobName}: start={this.startSegment} end={this.endSegment}");
}
catch
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.StartAsync failed");
throw;
}
}
/// <summary>
/// Is called on exceptions, if non-null; can be set by application
/// </summary>
@ -156,6 +194,55 @@ namespace DurableTask.Netherite.Faster
return $"{this.blobName}.{segmentId}";
}
internal void CheckForHangs(object _)
{
DateTime threshold = DateTime.UtcNow - this.limit;
foreach(var kvp in this.pendingReadWriteOperations)
{
if (kvp.Value.TimeStamp < threshold)
{
this.BlobManager.PartitionErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false);
return;
}
}
foreach (var kvp in this.pendingRemoveOperations)
{
if (kvp.Value.TimeStamp < threshold)
{
this.BlobManager.PartitionErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false);
return;
}
}
}
void CancelAllRequests()
{
foreach (var id in this.pendingReadWriteOperations.Keys.ToList())
{
if (this.pendingReadWriteOperations.TryRemove(id, out var request))
{
if (request.IsRead)
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.ReadAsync id={id} (Canceled)");
}
else
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.WriteAsync id={id} (Canceled)");
}
request.Callback(uint.MaxValue, request.NumBytes, request.Context);
}
}
foreach (var id in this.pendingRemoveOperations.Keys.ToList())
{
if (this.pendingRemoveOperations.TryRemove(id, out var request))
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.RemoveSegmentAsync id={id} (Canceled)");
request.Callback(request.Result);
}
}
}
//---- the overridden methods represent the interface for a generic storage device
/// <summary>
@ -163,7 +250,7 @@ namespace DurableTask.Netherite.Faster
/// </summary>
public override void Dispose()
{
// there are no resources that need to be freed
this.hangCheckTimer.Dispose();
}
/// <summary>
@ -174,10 +261,23 @@ namespace DurableTask.Netherite.Faster
/// <param name="result"></param>
public override void RemoveSegmentAsync(int segment, AsyncCallback callback, IAsyncResult result)
{
long id = Interlocked.Increment(ref AzureStorageDevice.sequenceNumber);
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.RemoveSegmentAsync id={id} segment={segment}");
this.pendingRemoveOperations.TryAdd(id, new RemoveRequestInfo()
{
Callback = callback,
Result = result,
TimeStamp = DateTime.UtcNow
});
Task deletionTask = Task.CompletedTask;
if (this.blobs.TryRemove(segment, out BlobEntry entry))
{
CloudPageBlob pageBlob = entry.PageBlob;
Task deletionTask = this.BlobManager.PerformWithRetriesAsync(
deletionTask = this.BlobManager.PerformWithRetriesAsync(
null,
this.underLease,
"CloudPageBlob.DeleteAsync",
@ -191,9 +291,16 @@ namespace DurableTask.Netherite.Faster
await pageBlob.DeleteAsync(cancellationToken: this.PartitionErrorHandler.Token);
return 1;
});
deletionTask.ContinueWith((Task t) => callback(result));
}
deletionTask.ContinueWith((Task t) =>
{
if (this.pendingRemoveOperations.TryRemove(id, out var request))
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.RemoveSegmentAsync id={id}");
request.Callback(request.Result);
}
});
}
/// <summary>
@ -216,7 +323,7 @@ namespace DurableTask.Netherite.Faster
false,
async (numAttempts) =>
{
await pageBlob.DeleteAsync(cancellationToken: this.PartitionErrorHandler.Token);
await pageBlob.DeleteIfExistsAsync(cancellationToken: this.PartitionErrorHandler.Token);
return 1;
});
}
@ -229,7 +336,18 @@ namespace DurableTask.Netherite.Faster
/// </summary>
public override unsafe void ReadAsync(int segmentId, ulong sourceAddress, IntPtr destinationAddress, uint readLength, DeviceIOCompletionCallback callback, object context)
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"AzureStorageDevice.ReadAsync Called segmentId={segmentId} sourceAddress={sourceAddress} readLength={readLength}");
long id = Interlocked.Increment(ref AzureStorageDevice.sequenceNumber);
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.ReadAsync id={id} segmentId={segmentId} sourceAddress={sourceAddress} readLength={readLength}");
this.pendingReadWriteOperations.TryAdd(id, new ReadWriteRequestInfo()
{
IsRead = true,
Callback = callback,
NumBytes = readLength,
Context = context,
TimeStamp = DateTime.UtcNow
});
// It is up to the allocator to make sure no reads are issued to segments before they are written
if (!this.blobs.TryGetValue(segmentId, out BlobEntry blobEntry))
@ -240,18 +358,21 @@ namespace DurableTask.Netherite.Faster
throw exception;
}
this.ReadFromBlobUnsafeAsync(blobEntry.PageBlob, (long)sourceAddress, (long)destinationAddress, readLength)
this.ReadFromBlobUnsafeAsync(blobEntry.PageBlob, (long)sourceAddress, (long)destinationAddress, readLength, id)
.ContinueWith((Task t) =>
{
if (t.IsFaulted)
if (this.pendingReadWriteOperations.TryRemove(id, out ReadWriteRequestInfo request))
{
this.BlobManager?.StorageTracer?.FasterStorageProgress("AzureStorageDevice.ReadAsync Returned (Failure)");
callback(uint.MaxValue, readLength, context);
}
else
{
this.BlobManager?.StorageTracer?.FasterStorageProgress("AzureStorageDevice.ReadAsync Returned");
callback(0, readLength, context);
if (t.IsFaulted)
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.ReadAsync id={id} (Failure)");
request.Callback(uint.MaxValue, request.NumBytes, request.Context);
}
else
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.ReadAsync id={id}");
request.Callback(0, request.NumBytes, request.Context);
}
}
});
}
@ -261,7 +382,18 @@ namespace DurableTask.Netherite.Faster
/// </summary>
public override void WriteAsync(IntPtr sourceAddress, int segmentId, ulong destinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context)
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"AzureStorageDevice.WriteAsync Called segmentId={segmentId} destinationAddress={destinationAddress} numBytesToWrite={numBytesToWrite}");
long id = Interlocked.Increment(ref AzureStorageDevice.sequenceNumber);
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.WriteAsync id={id} segmentId={segmentId} destinationAddress={destinationAddress} numBytesToWrite={numBytesToWrite}");
this.pendingReadWriteOperations.TryAdd(id, new ReadWriteRequestInfo()
{
IsRead = false,
Callback = callback,
NumBytes = numBytesToWrite,
Context = context,
TimeStamp = DateTime.UtcNow
});
if (!this.blobs.TryGetValue(segmentId, out BlobEntry blobEntry))
{
@ -280,17 +412,17 @@ namespace DurableTask.Netherite.Faster
// Otherwise, some other thread beat us to it. Okay to use their blobs.
blobEntry = this.blobs[segmentId];
}
this.TryWriteAsync(blobEntry, sourceAddress, destinationAddress, numBytesToWrite, callback, context);
this.TryWriteAsync(blobEntry, sourceAddress, destinationAddress, numBytesToWrite, id);
}
//---- The actual read and write accesses to the page blobs
unsafe Task WritePortionToBlobUnsafeAsync(CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, long offset, uint length)
unsafe Task WritePortionToBlobUnsafeAsync(CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, long offset, uint length, long id)
{
return this.WritePortionToBlobAsync(new UnmanagedMemoryStream((byte*)sourceAddress + offset, length), blob, sourceAddress, destinationAddress, offset, length);
return this.WritePortionToBlobAsync(new UnmanagedMemoryStream((byte*)sourceAddress + offset, length), blob, sourceAddress, destinationAddress, offset, length, id);
}
async Task WritePortionToBlobAsync(UnmanagedMemoryStream stream, CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, long offset, uint length)
async Task WritePortionToBlobAsync(UnmanagedMemoryStream stream, CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, long offset, uint length, long id)
{
using (stream)
{
@ -300,7 +432,7 @@ namespace DurableTask.Netherite.Faster
true,
"CloudPageBlob.WritePagesAsync",
"WriteToDevice",
$"length={length} destinationAddress={destinationAddress + offset}",
$"id={id} length={length} destinationAddress={destinationAddress + offset}",
blob.Name,
1000 + (int)length / 1000,
true,
@ -316,8 +448,7 @@ namespace DurableTask.Netherite.Faster
var blobRequestOptions = numAttempts > 2 ? BlobManager.BlobRequestOptionsDefault : BlobManager.BlobRequestOptionsAggressiveTimeout;
await blob.WritePagesAsync(stream, destinationAddress + offset,
contentChecksum: null, accessCondition: null, options: blobRequestOptions, operationContext: null, cancellationToken: this.PartitionErrorHandler.Token)
.ConfigureAwait(false);
contentChecksum: null, accessCondition: null, options: blobRequestOptions, operationContext: null, cancellationToken: this.PartitionErrorHandler.Token);
}
return (long)length;
@ -325,12 +456,12 @@ namespace DurableTask.Netherite.Faster
}
}
unsafe Task ReadFromBlobUnsafeAsync(CloudPageBlob blob, long sourceAddress, long destinationAddress, uint readLength)
unsafe Task ReadFromBlobUnsafeAsync(CloudPageBlob blob, long sourceAddress, long destinationAddress, uint readLength, long id)
{
return this.ReadFromBlobAsync(new UnmanagedMemoryStream((byte*)destinationAddress, readLength, readLength, FileAccess.Write), blob, sourceAddress, readLength);
return this.ReadFromBlobAsync(new UnmanagedMemoryStream((byte*)destinationAddress, readLength, readLength, FileAccess.Write), blob, sourceAddress, readLength, id);
}
async Task ReadFromBlobAsync(UnmanagedMemoryStream stream, CloudPageBlob blob, long sourceAddress, uint readLength)
async Task ReadFromBlobAsync(UnmanagedMemoryStream stream, CloudPageBlob blob, long sourceAddress, uint readLength, long id)
{
using (stream)
{
@ -344,7 +475,7 @@ namespace DurableTask.Netherite.Faster
true,
"CloudPageBlob.DownloadRangeToStreamAsync",
"ReadFromDevice",
$"readLength={length} sourceAddress={sourceAddress + offset}",
$"id={id} readLength={length} sourceAddress={sourceAddress + offset}",
blob.Name,
1000 + (int)length / 1000,
true,
@ -361,8 +492,7 @@ namespace DurableTask.Netherite.Faster
? BlobManager.BlobRequestOptionsDefault : BlobManager.BlobRequestOptionsAggressiveTimeout;
await blob
.DownloadRangeToStreamAsync(stream, sourceAddress + offset, length, accessCondition: null, options: blobRequestOptions, operationContext: null, cancellationToken: this.PartitionErrorHandler.Token)
.ConfigureAwait(false);
.DownloadRangeToStreamAsync(stream, sourceAddress + offset, length, accessCondition: null, options: blobRequestOptions, operationContext: null, cancellationToken: this.PartitionErrorHandler.Token);
}
if (stream.Position != offset + length)
@ -379,43 +509,46 @@ namespace DurableTask.Netherite.Faster
}
}
void TryWriteAsync(BlobEntry blobEntry, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context)
void TryWriteAsync(BlobEntry blobEntry, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, long id)
{
// If pageBlob is null, it is being created. Attempt to queue the write for the creator to complete after it is done
if (blobEntry.PageBlob == null
&& blobEntry.TryQueueAction(p => this.WriteToBlobAsync(p, sourceAddress, destinationAddress, numBytesToWrite, callback, context)))
&& blobEntry.TryQueueAction(p => this.WriteToBlobAsync(p, sourceAddress, destinationAddress, numBytesToWrite, id)))
{
return;
}
// Otherwise, invoke directly.
this.WriteToBlobAsync(blobEntry.PageBlob, sourceAddress, destinationAddress, numBytesToWrite, callback, context);
this.WriteToBlobAsync(blobEntry.PageBlob, sourceAddress, destinationAddress, numBytesToWrite, id);
}
unsafe void WriteToBlobAsync(CloudPageBlob blob, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context)
unsafe void WriteToBlobAsync(CloudPageBlob blob, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, long id)
{
this.WriteToBlobAsync(blob, sourceAddress, (long)destinationAddress, numBytesToWrite)
this.WriteToBlobAsync(blob, sourceAddress, (long)destinationAddress, numBytesToWrite, id)
.ContinueWith((Task t) =>
{
if (t.IsFaulted)
if (this.pendingReadWriteOperations.TryRemove(id, out ReadWriteRequestInfo request))
{
this.BlobManager?.StorageTracer?.FasterStorageProgress("AzureStorageDevice.WriteAsync Returned (Failure)");
callback(uint.MaxValue, numBytesToWrite, context);
}
else
{
this.BlobManager?.StorageTracer?.FasterStorageProgress("AzureStorageDevice.WriteAsync Returned");
callback(0, numBytesToWrite, context);
if (t.IsFaulted)
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.WriteAsync id={id} (Failure)");
request.Callback(uint.MaxValue, request.NumBytes, request.Context);
}
else
{
this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.WriteAsync id={id}");
request.Callback(0, request.NumBytes, request.Context);
}
}
});
}
async Task WriteToBlobAsync(CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, uint numBytesToWrite)
async Task WriteToBlobAsync(CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, uint numBytesToWrite, long id)
{
long offset = 0;
while (numBytesToWrite > 0)
{
var length = Math.Min(numBytesToWrite, MAX_UPLOAD_SIZE);
await this.WritePortionToBlobUnsafeAsync(blob, sourceAddress, destinationAddress, offset, length).ConfigureAwait(false);
await this.WritePortionToBlobUnsafeAsync(blob, sourceAddress, destinationAddress, offset, length, id);
numBytesToWrite -= length;
offset += length;
}

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

@ -70,8 +70,8 @@ namespace DurableTask.Netherite.Faster
volatile System.Diagnostics.Stopwatch leaseTimer;
internal const long HashTableSize = 1L << 14; // 16 k buckets, 1 GB
//internal const long HashTableSize = 1L << 14; // 8 M buckets, 512 GB
internal const long HashTableSize = 1L << 14; // 16 k buckets, 1 MB
internal const long HashTableSizeBytes = HashTableSize * 64;
public class FasterTuningParameters
{
@ -82,45 +82,70 @@ namespace DurableTask.Netherite.Faster
public int? StoreLogSegmentSizeBits;
public int? StoreLogMemorySizeBits;
public double? StoreLogMutableFraction;
public int? ExpectedObjectSize;
public int? NumPagesToPreload;
}
public FasterLogSettings GetDefaultEventLogSettings(bool useSeparatePageBlobStorage, FasterTuningParameters tuningParameters) => new FasterLogSettings
{
LogDevice = this.EventLogDevice,
LogCommitManager = this.UseLocalFiles
? new LocalLogCommitManager($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{CommitBlobName}")
? null // TODO: fix this: new LocalLogCommitManager($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{CommitBlobName}")
: (ILogCommitManager)this,
PageSizeBits = tuningParameters?.EventLogPageSizeBits ?? 21, // 2MB
SegmentSizeBits = tuningParameters?.EventLogSegmentSizeBits ??
(useSeparatePageBlobStorage ? 35 // 32 GB
: 30), // 1 GB
: 26), // 64 MB
MemorySizeBits = tuningParameters?.EventLogMemorySizeBits ?? 22, // 2MB
};
public LogSettings GetDefaultStoreLogSettings(bool useSeparatePageBlobStorage, uint numPartitions, FasterTuningParameters tuningParameters) => new LogSettings
public LogSettings GetDefaultStoreLogSettings(
bool useSeparatePageBlobStorage,
long upperBoundOnAvailable,
FasterTuningParameters tuningParameters)
{
LogDevice = this.HybridLogDevice,
ObjectLogDevice = this.ObjectLogDevice,
PageSizeBits = tuningParameters?.StoreLogPageSizeBits ?? 17, // 128kB
MutableFraction = tuningParameters?.StoreLogMutableFraction ?? 0.9,
SegmentSizeBits = tuningParameters?.StoreLogSegmentSizeBits ??
(useSeparatePageBlobStorage ? 35 // 32 GB
: 32), // 4 GB
CopyReadsToTail = CopyReadsToTail.FromReadOnly,
MemorySizeBits = tuningParameters?.StoreLogMemorySizeBits ??
((numPartitions <= 1) ? 25 : // 32MB
(numPartitions <= 2) ? 24 : // 16MB
(numPartitions <= 4) ? 23 : // 8MB
(numPartitions <= 8) ? 22 : // 4MB
(numPartitions <= 16) ? 21 : // 2MB
20), // 1MB
};
var settings = new LogSettings
{
LogDevice = this.HybridLogDevice,
ObjectLogDevice = this.ObjectLogDevice,
PageSizeBits = tuningParameters?.StoreLogPageSizeBits ?? 10, // 1kB
MutableFraction = tuningParameters?.StoreLogMutableFraction ?? 0.9,
SegmentSizeBits = tuningParameters?.StoreLogSegmentSizeBits ??
(useSeparatePageBlobStorage ? 35 // 32 GB
: 32), // 4 GB
CopyReadsToTail = CopyReadsToTail.FromReadOnly,
};
// compute a reasonable memory size for the log considering maximally availablee memory, and expansion factor
if (tuningParameters?.StoreLogMemorySizeBits != null)
{
settings.MemorySizeBits = tuningParameters.StoreLogMemorySizeBits.Value;
}
else
{
double expansionFactor = (24 + ((double)(tuningParameters?.ExpectedObjectSize ?? 216))) / 24;
long estimate = (long)(upperBoundOnAvailable / expansionFactor);
int memorybits = 0;
while (estimate > 0)
{
memorybits++;
estimate >>= 1;
}
memorybits = Math.Max(settings.PageSizeBits + 2, memorybits); // do not use less than 4 pages
settings.MemorySizeBits = memorybits;
}
return settings;
}
static readonly int[] StorageFormatVersion = new int[] {
1, //initial version
2, //0.7.0-beta changed singleton storage, and adds dequeue count
3, //changed organization of files
};
4, //use Faster v2, reduced page size
};
public static string GetStorageFormat(NetheriteOrchestrationServiceSettings settings)
{
@ -198,7 +223,6 @@ namespace DurableTask.Netherite.Faster
public CheckpointSettings StoreCheckpointSettings => new CheckpointSettings
{
CheckpointManager = this.UseLocalFiles ? (ICheckpointManager)this.LocalCheckpointManager : (ICheckpointManager)this,
CheckPointType = CheckpointType.FoldOver
};
public const int MaxRetries = 10;
@ -382,7 +406,18 @@ namespace DurableTask.Netherite.Faster
}
}
internal void ClosePSFDevices() => Array.ForEach(this.PsfLogDevices, logDevice => logDevice.Dispose());
internal void DisposeDevices()
{
Dispose(this.HybridLogDevice);
Dispose(this.ObjectLogDevice);
Array.ForEach(this.PsfLogDevices, logDevice => Dispose(logDevice));
void Dispose(IDevice device)
{
this.TraceHelper.FasterStorageProgress($"Disposing Device {device.FileName}");
device.Dispose();
}
}
public void HandleStorageError(string where, string message, string blobName, Exception e, bool isFatal, bool isWarning)
{
@ -506,7 +541,7 @@ namespace DurableTask.Netherite.Faster
accessCondition: null,
options: BlobManager.BlobRequestOptionsDefault,
operationContext: null,
cancellationToken: this.PartitionErrorHandler.Token).ConfigureAwait(false);
cancellationToken: this.PartitionErrorHandler.Token);
this.TraceHelper.LeaseAcquired();
}
@ -523,7 +558,7 @@ namespace DurableTask.Netherite.Faster
// the previous owner has not released the lease yet,
// try again until it becomes available, should be relatively soon
// as the transport layer is supposed to shut down the previous owner when starting this
await Task.Delay(TimeSpan.FromSeconds(1), this.PartitionErrorHandler.Token).ConfigureAwait(false);
await Task.Delay(TimeSpan.FromSeconds(1), this.PartitionErrorHandler.Token);
continue;
}
@ -565,7 +600,7 @@ namespace DurableTask.Netherite.Faster
continue;
}
}
catch (StorageException ex) when (numAttempts < BlobManager.MaxRetries && BlobUtils.IsTransientStorageError(ex, this.PartitionErrorHandler.Token))
catch (Exception ex) when (numAttempts < BlobManager.MaxRetries && BlobUtils.IsTransientStorageError(ex, this.PartitionErrorHandler.Token))
{
if (BlobUtils.IsTimeout(ex))
{
@ -605,8 +640,7 @@ namespace DurableTask.Netherite.Faster
{
this.TraceHelper.LeaseProgress($"Renewing lease at {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s");
this.FaultInjector?.StorageAccess(this, "RenewLeaseAsync", "RenewLease", this.eventLogCommitBlob.Name);
await this.eventLogCommitBlob.RenewLeaseAsync(acc, this.PartitionErrorHandler.Token)
.ConfigureAwait(false);
await this.eventLogCommitBlob.RenewLeaseAsync(acc, this.PartitionErrorHandler.Token);
this.TraceHelper.LeaseRenewed(this.leaseTimer.Elapsed.TotalSeconds, this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds);
if (nextLeaseTimer.ElapsedMilliseconds > 2000)
@ -647,7 +681,7 @@ namespace DurableTask.Netherite.Faster
}
// wait for successful renewal, or exit the loop as this throws
await this.NextLeaseRenewalTask.ConfigureAwait(false);
await this.NextLeaseRenewalTask;
}
}
catch (OperationCanceledException)
@ -692,7 +726,7 @@ namespace DurableTask.Netherite.Faster
accessCondition: acc,
options: BlobManager.BlobRequestOptionsDefault,
operationContext: null,
cancellationToken: this.PartitionErrorHandler.Token).ConfigureAwait(false);
cancellationToken: this.PartitionErrorHandler.Token);
this.TraceHelper.LeaseReleased(this.leaseTimer.Elapsed.TotalSeconds);
}
@ -871,47 +905,55 @@ namespace DurableTask.Netherite.Faster
#region ILogCommitManager
void ILogCommitManager.Commit(long beginAddress, long untilAddress, byte[] commitMetadata)
void ILogCommitManager.Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum)
{
this.StorageTracer?.FasterStorageProgress($"ILogCommitManager.Commit Called beginAddress={beginAddress} untilAddress={untilAddress}");
try
{
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ILogCommitManager.Commit beginAddress={beginAddress} untilAddress={untilAddress}");
AccessCondition acc = new AccessCondition() { LeaseId = this.leaseId };
AccessCondition acc = new AccessCondition() { LeaseId = this.leaseId };
this.PerformWithRetries(
false,
"CloudBlockBlob.UploadFromByteArray",
"WriteCommitLogMetadata",
"",
this.eventLogCommitBlob.Name,
1000,
true,
(int numAttempts) =>
{
try
this.PerformWithRetries(
false,
"CloudBlockBlob.UploadFromByteArray",
"WriteCommitLogMetadata",
"",
this.eventLogCommitBlob.Name,
1000,
true,
(int numAttempts) =>
{
var blobRequestOptions = numAttempts > 2 ? BlobManager.BlobRequestOptionsDefault : BlobManager.BlobRequestOptionsAggressiveTimeout;
this.eventLogCommitBlob.UploadFromByteArray(commitMetadata, 0, commitMetadata.Length, acc, blobRequestOptions);
return (commitMetadata.Length, true);
}
catch (StorageException ex) when (BlobUtils.LeaseConflict(ex))
{
// We lost the lease to someone else. Terminate ownership immediately.
this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(ILogCommitManager.Commit));
this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not commit because of lost lease", this.eventLogCommitBlob?.Name, ex, true, this.PartitionErrorHandler.IsTerminated);
throw;
}
catch (StorageException ex) when (BlobUtils.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries)
{
// if we get here, the lease renewal task did not complete in time
// give it another chance to complete
this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: wait for next renewal");
this.NextLeaseRenewalTask.Wait();
this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete");
return (commitMetadata.Length, false);
}
});
this.StorageTracer?.FasterStorageProgress($"ILogCommitManager.Commit Returned");
try
{
var blobRequestOptions = numAttempts > 2 ? BlobManager.BlobRequestOptionsDefault : BlobManager.BlobRequestOptionsAggressiveTimeout;
this.eventLogCommitBlob.UploadFromByteArray(commitMetadata, 0, commitMetadata.Length, acc, blobRequestOptions);
return (commitMetadata.Length, true);
}
catch (StorageException ex) when (BlobUtils.LeaseConflict(ex))
{
// We lost the lease to someone else. Terminate ownership immediately.
this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(ILogCommitManager.Commit));
this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not commit because of lost lease", this.eventLogCommitBlob?.Name, ex, true, this.PartitionErrorHandler.IsTerminated);
throw;
}
catch (StorageException ex) when (BlobUtils.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries)
{
// if we get here, the lease renewal task did not complete in time
// give it another chance to complete
this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: wait for next renewal");
this.NextLeaseRenewalTask.Wait();
this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete");
return (commitMetadata.Length, false);
}
});
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.Commit");
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.Commit failed");
throw;
}
}
@ -921,54 +963,77 @@ namespace DurableTask.Netherite.Faster
yield return 0;
}
void ILogCommitManager.OnRecovery(long commitNum)
{
// TODO: make sure our use of single commmit is safe
}
void ILogCommitManager.RemoveAllCommits()
{
// TODO: make sure our use of single commmit is safe
}
void ILogCommitManager.RemoveCommit(long commitNum)
{
// TODO: make sure our use of single commmit is safe
}
byte[] ILogCommitManager.GetCommitMetadata(long commitNum)
{
this.StorageTracer?.FasterStorageProgress($"ILogCommitManager.GetCommitMetadata Called (thread={Thread.CurrentThread.ManagedThreadId})");
AccessCondition acc = new AccessCondition() { LeaseId = this.leaseId };
using var stream = new MemoryStream();
try
{
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ILogCommitManager.GetCommitMetadata (thread={Thread.CurrentThread.ManagedThreadId})");
AccessCondition acc = new AccessCondition() { LeaseId = this.leaseId };
using var stream = new MemoryStream();
this.PerformWithRetries(
false,
"CloudBlockBlob.DownloadToStream",
"ReadCommitLogMetadata",
"",
this.eventLogCommitBlob.Name,
1000,
true,
(int numAttempts) =>
{
if (numAttempts > 0)
this.PerformWithRetries(
false,
"CloudBlockBlob.DownloadToStream",
"ReadCommitLogMetadata",
"",
this.eventLogCommitBlob.Name,
1000,
true,
(int numAttempts) =>
{
stream.Seek(0, SeekOrigin.Begin);
}
if (numAttempts > 0)
{
stream.Seek(0, SeekOrigin.Begin);
}
try
{
var blobRequestOptions = numAttempts > 2 ? BlobManager.BlobRequestOptionsDefault : BlobManager.BlobRequestOptionsAggressiveTimeout;
this.eventLogCommitBlob.DownloadToStream(stream, acc, blobRequestOptions);
return (stream.Position, true);
}
catch (StorageException ex) when (BlobUtils.LeaseConflict(ex))
{
try
{
var blobRequestOptions = numAttempts > 2 ? BlobManager.BlobRequestOptionsDefault : BlobManager.BlobRequestOptionsAggressiveTimeout;
this.eventLogCommitBlob.DownloadToStream(stream, acc, blobRequestOptions);
return (stream.Position, true);
}
catch (StorageException ex) when (BlobUtils.LeaseConflict(ex))
{
// We lost the lease to someone else. Terminate ownership immediately.
this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(ILogCommitManager.GetCommitMetadata));
this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not read latest commit due to lost lease", this.eventLogCommitBlob?.Name, ex, true, this.PartitionErrorHandler.IsTerminated);
throw;
}
catch (StorageException ex) when (BlobUtils.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries)
{
this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not read latest commit due to lost lease", this.eventLogCommitBlob?.Name, ex, true, this.PartitionErrorHandler.IsTerminated);
throw;
}
catch (StorageException ex) when (BlobUtils.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries)
{
// if we get here, the lease renewal task did not complete in time
// give it another chance to complete
this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: wait for next renewal");
this.NextLeaseRenewalTask.Wait();
this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete");
return (0, false);
}
});
this.NextLeaseRenewalTask.Wait();
this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete");
return (0, false);
}
});
var bytes = stream.ToArray();
this.StorageTracer?.FasterStorageProgress($"ILogCommitManager.GetCommitMetadata Returned {bytes?.Length ?? null} bytes");
return bytes.Length == 0 ? null : bytes;
var bytes = stream.ToArray();
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.GetCommitMetadata {bytes?.Length ?? null} bytes");
return bytes.Length == 0 ? null : bytes;
}
catch(Exception e)
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.GetCommitMetadata failed with {e.GetType().Name}: {e.Message}");
throw;
}
}
#endregion
@ -993,7 +1058,7 @@ namespace DurableTask.Netherite.Faster
void ICheckpointManager.CommitLogCheckpoint(Guid logToken, byte[] commitMetadata)
=> this.CommitLogCheckpoint(logToken, commitMetadata, InvalidPsfGroupOrdinal);
void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, int version, byte[] commitMetadata, DeltaLog deltaLog)
void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, long version, byte[] commitMetadata, DeltaLog deltaLog)
=> this.CommitLogIncrementalCheckpoint(logToken, version, commitMetadata, deltaLog, InvalidPsfGroupOrdinal);
byte[] ICheckpointManager.GetIndexCheckpointMetadata(Guid indexToken)
@ -1017,14 +1082,14 @@ namespace DurableTask.Netherite.Faster
IEnumerable<Guid> ICheckpointManager.GetIndexCheckpointTokens()
{
var indexToken = this.CheckpointInfo.IndexToken;
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetIndexCheckpointTokens returned indexToken={indexToken}");
this.StorageTracer?.FasterStorageProgress($"StorageOp ICheckpointManager.GetIndexCheckpointTokens indexToken={indexToken}");
yield return indexToken;
}
IEnumerable<Guid> ICheckpointManager.GetLogCheckpointTokens()
{
var logToken = this.CheckpointInfo.LogToken;
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetLogCheckpointTokens returned logToken={logToken}");
this.StorageTracer?.FasterStorageProgress($"StorageOp ICheckpointManager.GetLogCheckpointTokens logToken={logToken}");
yield return logToken;
}
@ -1080,126 +1145,159 @@ namespace DurableTask.Netherite.Faster
internal void CommitIndexCheckpoint(Guid indexToken, byte[] commitMetadata, int psfGroupOrdinal)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.CommitIndexCheckpoint Called on {tag}, indexToken={indexToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetIndexCheckpointMetaBlobName(indexToken));
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.CommitIndexCheckpoint on {tag}, indexToken={indexToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetIndexCheckpointMetaBlobName(indexToken));
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenWrite",
"WriteIndexCheckpointMetadata",
$"token={indexToken} size={commitMetadata.Length}",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using (var blobStream = metaFileBlob.OpenWrite())
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenWrite",
"WriteIndexCheckpointMetadata",
$"token={indexToken} size={commitMetadata.Length}",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using var writer = new BinaryWriter(blobStream);
writer.Write(commitMetadata.Length);
writer.Write(commitMetadata);
writer.Flush();
return (commitMetadata.Length, true);
}
});
using (var blobStream = metaFileBlob.OpenWrite())
{
using var writer = new BinaryWriter(blobStream);
writer.Write(commitMetadata.Length);
writer.Write(commitMetadata);
writer.Flush();
return (commitMetadata.Length, true);
}
});
(isPsf ? this.PsfCheckpointInfos[psfGroupOrdinal] : this.CheckpointInfo).IndexToken = indexToken;
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.CommitIndexCheckpoint Returned from {tag}, target={metaFileBlob.Name}");
(isPsf ? this.PsfCheckpointInfos[psfGroupOrdinal] : this.CheckpointInfo).IndexToken = indexToken;
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitIndexCheckpoint from {tag}, target={metaFileBlob.Name}");
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitIndexCheckpoint failed");
throw;
}
}
internal void CommitLogCheckpoint(Guid logToken, byte[] commitMetadata, int psfGroupOrdinal)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.CommitLogCheckpoint Called on {tag}, logToken={logToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetHybridLogCheckpointMetaBlobName(logToken));
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.CommitLogCheckpoint on {tag}, logToken={logToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetHybridLogCheckpointMetaBlobName(logToken));
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenWrite",
"WriteHybridLogCheckpointMetadata",
$"token={logToken}",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using (var blobStream = metaFileBlob.OpenWrite())
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenWrite",
"WriteHybridLogCheckpointMetadata",
$"token={logToken}",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using var writer = new BinaryWriter(blobStream);
writer.Write(commitMetadata.Length);
writer.Write(commitMetadata);
writer.Flush();
return (commitMetadata.Length + 4, true);
}
});
using (var blobStream = metaFileBlob.OpenWrite())
{
using var writer = new BinaryWriter(blobStream);
writer.Write(commitMetadata.Length);
writer.Write(commitMetadata);
writer.Flush();
return (commitMetadata.Length + 4, true);
}
});
(isPsf ? this.PsfCheckpointInfos[psfGroupOrdinal] : this.CheckpointInfo).LogToken = logToken;
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.CommitLogCheckpoint Returned from {tag}, target={metaFileBlob.Name}");
(isPsf ? this.PsfCheckpointInfos[psfGroupOrdinal] : this.CheckpointInfo).LogToken = logToken;
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitLogCheckpoint from {tag}, target={metaFileBlob.Name}");
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitLogCheckpoint failed");
throw;
}
}
internal void CommitLogIncrementalCheckpoint(Guid logToken, int version, byte[] commitMetadata, DeltaLog deltaLog, int indexOrdinal)
internal void CommitLogIncrementalCheckpoint(Guid logToken, long version, byte[] commitMetadata, DeltaLog deltaLog, int indexOrdinal)
{
throw new NotImplementedException("incremental checkpointing is not implemented");
}
internal byte[] GetIndexCheckpointMetadata(Guid indexToken, int psfGroupOrdinal)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetIndexCommitMetadata Called on {tag}, indexToken={indexToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetIndexCheckpointMetaBlobName(indexToken));
byte[] result = null;
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetIndexCommitMetadata on {tag}, indexToken={indexToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetIndexCheckpointMetaBlobName(indexToken));
byte[] result = null;
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenRead",
"ReadIndexCheckpointMetadata",
"",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using var blobstream = metaFileBlob.OpenRead();
using var reader = new BinaryReader(blobstream);
var len = reader.ReadInt32();
result = reader.ReadBytes(len);
return (len + 4, true);
});
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenRead",
"ReadIndexCheckpointMetadata",
"",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using var blobstream = metaFileBlob.OpenRead();
using var reader = new BinaryReader(blobstream);
var len = reader.ReadInt32();
result = reader.ReadBytes(len);
return (len + 4, true);
});
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetIndexCommitMetadata Returned {result?.Length ?? null} bytes from {tag}, target={metaFileBlob.Name}");
return result;
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCommitMetadata {result?.Length ?? null} bytes from {tag}, target={metaFileBlob.Name}");
return result;
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCommitMetadata failed");
throw;
}
}
internal byte[] GetLogCheckpointMetadata(Guid logToken, int psfGroupOrdinal, DeltaLog deltaLog, bool scanDelta, long recoverTo)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetIndexCommitMetadata Called on {tag}, logToken={logToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetHybridLogCheckpointMetaBlobName(logToken));
byte[] result = null;
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetIndexCommitMetadata on {tag}, logToken={logToken}");
var partDir = isPsf ? this.blockBlobPartitionDirectory.GetDirectoryReference(this.PsfGroupFolderName(psfGroupOrdinal)) : this.blockBlobPartitionDirectory;
var metaFileBlob = partDir.GetBlockBlobReference(this.GetHybridLogCheckpointMetaBlobName(logToken));
byte[] result = null;
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenRead",
"ReadLogCheckpointMetadata",
"",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using var blobstream = metaFileBlob.OpenRead();
using var reader = new BinaryReader(blobstream);
var len = reader.ReadInt32();
result = reader.ReadBytes(len);
return (len + 4, true);
});
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetIndexCommitMetadata Returned {result?.Length ?? null} bytes from {tag}, target={metaFileBlob.Name}");
return result;
this.PerformWithRetries(
false,
"CloudBlockBlob.OpenRead",
"ReadLogCheckpointMetadata",
"",
metaFileBlob.Name,
1000,
true,
(numAttempts) =>
{
using var blobstream = metaFileBlob.OpenRead();
using var reader = new BinaryReader(blobstream);
var len = reader.ReadInt32();
result = reader.ReadBytes(len);
return (len + 4, true);
});
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCommitMetadata {result?.Length ?? null} bytes from {tag}, target={metaFileBlob.Name}");
return result;
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCommitMetadata failed");
throw;
}
}
void GetPartitionDirectories(bool isPsf, int psfGroupOrdinal, string path, out CloudBlobDirectory blockBlobDir, out CloudBlobDirectory pageBlobDir)
@ -1212,53 +1310,85 @@ namespace DurableTask.Netherite.Faster
internal IDevice GetIndexDevice(Guid indexToken, int psfGroupOrdinal)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetIndexDevice Called on {tag}, indexToken={indexToken}");
var (path, blobName) = this.GetPrimaryHashTableBlobName(indexToken);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetIndexDevice Returned from {tag}, target={blockBlobDir.Prefix}{blobName}");
return device;
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetIndexDevice on {tag}, indexToken={indexToken}");
var (path, blobName) = this.GetPrimaryHashTableBlobName(indexToken);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexDevice from {tag}, target={blockBlobDir.Prefix}{blobName}");
return device;
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexDevice failed");
throw;
}
}
internal IDevice GetSnapshotLogDevice(Guid token, int psfGroupOrdinal)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetSnapshotLogDevice Called on {tag}, token={token}");
var (path, blobName) = this.GetLogSnapshotBlobName(token);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetSnapshotLogDevice Returned from {tag}, blobDirectory={blockBlobDir} blobName={blobName}");
return device;
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetSnapshotLogDevice on {tag}, token={token}");
var (path, blobName) = this.GetLogSnapshotBlobName(token);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotLogDevice from {tag}, blobDirectory={blockBlobDir} blobName={blobName}");
return device;
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotLogDevice failed");
throw;
}
}
internal IDevice GetSnapshotObjectLogDevice(Guid token, int psfGroupOrdinal)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetSnapshotObjectLogDevice Called on {tag}, token={token}");
var (path, blobName) = this.GetObjectLogSnapshotBlobName(token);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetSnapshotObjectLogDevice Returned from {tag}, blobDirectory={blockBlobDir} blobName={blobName}");
return device;
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetSnapshotObjectLogDevice on {tag}, token={token}");
var (path, blobName) = this.GetObjectLogSnapshotBlobName(token);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotObjectLogDevice from {tag}, blobDirectory={blockBlobDir} blobName={blobName}");
return device;
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotObjectLogDevice failed");
throw;
}
}
internal IDevice GetDeltaLogDevice(Guid token, int psfGroupOrdinal)
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetDeltaLogDevice Called on {tag}, token={token}");
var (path, blobName) = this.GetDeltaLogSnapshotBlobName(token);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"ICheckpointManager.GetDeltaLogDevice Returned from {tag}, blobDirectory={blockBlobDir} blobName={blobName}");
return device;
try
{
var (isPsf, tag) = this.IsPsfOrPrimary(psfGroupOrdinal);
this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetDeltaLogDevice on {tag}, token={token}");
var (path, blobName) = this.GetDeltaLogSnapshotBlobName(token);
this.GetPartitionDirectories(isPsf, psfGroupOrdinal, path, out var blockBlobDir, out var pageBlobDir);
var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation
device.StartAsync().Wait();
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetDeltaLogDevice from {tag}, blobDirectory={blockBlobDir} blobName={blobName}");
return device;
}
catch
{
this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetDeltaLogDevice failed");
throw;
}
}
#endregion
#endregion
internal async Task PersistSingletonsAsync(byte[] singletons, Guid guid)
{
@ -1331,7 +1461,7 @@ namespace DurableTask.Netherite.Faster
async Task writeBlob(CloudBlobDirectory partDir, string text)
{
var checkpointCompletedBlob = partDir.GetBlockBlobReference(this.GetCheckpointCompletedBlobName());
await this.ConfirmLeaseIsGoodForAWhileAsync().ConfigureAwait(false); // the lease protects the checkpoint completed file
await this.ConfirmLeaseIsGoodForAWhileAsync(); // the lease protects the checkpoint completed file
await this.PerformWithRetriesAsync(
BlobManager.AsynchronousStorageWriteMaxConcurrency,
true,

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

@ -12,13 +12,16 @@ namespace DurableTask.Netherite.Faster
class LocalFileCheckpointManager : ICheckpointManager
{
readonly CheckpointInfo checkpointInfo;
readonly LocalCheckpointManager localCheckpointManager;
readonly ICheckpointManager localCheckpointManager;
readonly string checkpointCompletedFilename;
internal LocalFileCheckpointManager(CheckpointInfo ci, string checkpointDir, string checkpointCompletedBlobName)
{
this.checkpointInfo = ci;
this.localCheckpointManager = new LocalCheckpointManager(checkpointDir);
this.localCheckpointManager = new DeviceLogCommitCheckpointManager
(new LocalStorageNamedDeviceFactory(),
new DefaultCheckpointNamingScheme(
new DirectoryInfo(checkpointDir).FullName), removeOutdated: true);
this.checkpointCompletedFilename = Path.Combine(checkpointDir, checkpointCompletedBlobName);
}
@ -40,7 +43,7 @@ namespace DurableTask.Netherite.Faster
this.checkpointInfo.LogToken = logToken;
}
void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, int version, byte[] commitMetadata, DeltaLog deltaLog)
void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, long version, byte[] commitMetadata, DeltaLog deltaLog)
{
throw new NotImplementedException("incremental checkpointing is not implemented");
}

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

@ -30,7 +30,7 @@ namespace DurableTask.Netherite.Faster
void ICheckpointManager.CommitLogCheckpoint(Guid logToken, byte[] commitMetadata)
=> this.blobManager.CommitLogCheckpoint(logToken, commitMetadata, this.groupOrdinal);
void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, int version, byte[] commitMetadata, DeltaLog deltaLog)
void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, long version, byte[] commitMetadata, DeltaLog deltaLog)
=> this.blobManager.CommitLogIncrementalCheckpoint(logToken, version, commitMetadata, deltaLog, this.groupOrdinal);
IDevice ICheckpointManager.GetIndexDevice(Guid indexToken)

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

@ -47,6 +47,8 @@ namespace DurableTask.Netherite.Faster
await this.ConfirmLeaseIsGoodForAWhileAsync().ConfigureAwait(false);
}
this.PartitionErrorHandler.Token.ThrowIfCancellationRequested();
this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}");
stopwatch.Restart();
@ -67,9 +69,10 @@ namespace DurableTask.Netherite.Faster
return;
}
catch (StorageException e) when (BlobUtils.IsTransientStorageError(e, this.PartitionErrorHandler.Token) && numAttempts < BlobManager.MaxRetries)
catch (Exception e) when (BlobUtils.IsTransientStorageError(e, this.PartitionErrorHandler.Token) && numAttempts < BlobManager.MaxRetries)
{
stopwatch.Stop();
if (BlobUtils.IsTimeout(e))
{
this.TraceHelper.FasterPerfWarning($"storage operation {name} ({intent}) timed out on attempt {numAttempts} after {stopwatch.Elapsed.TotalSeconds:F1}s, retrying now; target={target} {data}");
@ -147,7 +150,7 @@ namespace DurableTask.Netherite.Faster
return;
}
catch (StorageException e) when (numAttempts < BlobManager.MaxRetries
catch (Exception e) when (numAttempts < BlobManager.MaxRetries
&& BlobUtils.IsTransientStorageError(e, this.PartitionErrorHandler.Token))
{
stopwatch.Stop();

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

@ -8,22 +8,27 @@ namespace DurableTask.Netherite.Faster
using System.Collections.Generic;
using System.Linq;
using System.Text;
using System.Threading;
using System.Threading.Tasks;
using FASTER.core;
/// <summary>
/// Records cache and storage management traces for each object. This class is only used for testing and debugging, as it creates lots of overhead.
/// </summary>
class CacheDebugger
public class CacheDebugger
{
readonly TestHooks testHooks;
readonly ConcurrentDictionary<TrackedObjectKey, ObjectInfo> Objects = new ConcurrentDictionary<TrackedObjectKey, ObjectInfo>();
internal MemoryTracker MemoryTracker { get; set; }
public CacheDebugger(TestHooks testHooks)
{
this.testHooks = testHooks;
}
public bool EnableSizeChecking { get; set; } = true;
public enum CacheEvent
{
// reads and RMWs on the main session
@ -36,14 +41,26 @@ namespace DurableTask.Netherite.Faster
// Faster IFunctions
InitialUpdate,
PostInitialUpdate,
InPlaceUpdate,
CopyUpdate,
PostCopyUpdate,
SingleWriterUpsert,
SingleWriterCopyToTail,
SingleWriterCopyToReadCache,
SingleWriterCompaction,
PostSingleWriterUpsert,
PostSingleWriterCopyToTail,
PostSingleWriterCopyToReadCache,
PostSingleWriterCompaction,
SingleDeleter,
PostSingleDeleter,
ConcurrentWriter,
ConcurrentDeleter,
SingleReader,
SingleReaderPrefetch,
ConcurrentReader,
ConcurrentReaderPrefetch,
SingleWriter,
ConcurrentWriter,
// subscriptions to the FASTER log accessor
Evict,
@ -57,35 +74,79 @@ namespace DurableTask.Netherite.Faster
DeserializeBytes,
DeserializeObject,
// explicit failure
// tracking adjustment
TrackSize,
// other events
Fail,
Reset,
SizeCheckSuccess,
SizeCheckFail,
Faster,
};
public class ObjectInfo
{
public int CurrentVersion;
public List<Entry> CacheEvents;
public ConcurrentQueue<Entry> CacheEvents;
public long Size = 0;
public int? PendingRMW;
public override string ToString()
{
return $"Current=v{this.CurrentVersion} CacheEvents={this.CacheEvents.Count}";
return $"Current=v{this.CurrentVersion} Size={this.Size} CacheEvents={this.CacheEvents.Count}";
}
public string PrintCacheEvents() => string.Join(",", this.CacheEvents.Select(e => e.ToString()));
}
public Task CreateTimer(TimeSpan timeSpan)
{
return Task.Delay(timeSpan);
}
public async ValueTask CheckTiming(Task waitingFor, Task timer, string message)
{
var first = await Task.WhenAny(waitingFor, timer);
if (first == timer)
public string PrintCacheEvents()
{
this.testHooks.Error(this.GetType().Name, $"timeout: {message}");
var sb = new StringBuilder();
this.GetCacheEvents(out var events, out var entries);
foreach (var entry in events)
{
sb.Append(" ");
sb.Append(entry.ToString());
if (entry.CacheEvent == CacheEvent.TrackSize || entry.CacheEvent == CacheEvent.Reset)
{
sb.Append("=");
sb.Append(entries[entry.Address]);
}
}
return sb.ToString();
}
public void GetCacheEvents(out IList<Entry> entries, out SortedDictionary<long, long> entrySizes)
{
entries = new List<Entry>();
entrySizes = new SortedDictionary<long, long>();
foreach (var entry in this.CacheEvents)
{
if (entry.CacheEvent == CacheEvent.TrackSize)
{
entrySizes.TryGetValue(entry.Address, out long current);
entrySizes[entry.Address] = current + entry.Delta;
}
else if (entry.CacheEvent == CacheEvent.Reset)
{
entrySizes[entry.Address] = 0;
}
entries.Add(entry);
}
}
}
internal ObjectInfo GetObjectInfo(TrackedObjectKey key)
{
return this.Objects.AddOrUpdate(
key,
key => new ObjectInfo()
{
CacheEvents = new ConcurrentQueue<Entry>(),
},
(key, info) =>
{
return info;
});
}
public struct Entry
@ -93,50 +154,197 @@ namespace DurableTask.Netherite.Faster
public string EventId;
public CacheEvent CacheEvent;
public int? Version;
public long Delta;
public long Address;
public override string ToString()
{
var sb = new StringBuilder();
sb.Append(this.CacheEvent.ToString());
if (this.Version != null)
if (this.CacheEvent == CacheEvent.SizeCheckSuccess)
{
sb.Append('.');
sb.Append('v');
sb.Append(this.Version.ToString());
sb.Append('✓');
sb.Append(this.Delta);
}
else if (this.CacheEvent == CacheEvent.SizeCheckFail)
{
sb.Append("❌r");
sb.Append(this.Address);
sb.Append('a');
sb.Append(this.Delta);
}
else if (this.CacheEvent == CacheEvent.Faster)
{
sb.Append(this.EventId);
}
else
{
sb.Append(this.CacheEvent.ToString());
//if (!string.IsNullOrEmpty(this.EventId))
//{
// sb.Append('.');
// sb.Append(this.EventId);
//}
if (this.Version != null)
{
sb.Append('.');
sb.Append('v');
sb.Append(this.Version.ToString());
}
if (this.CacheEvent == CacheEvent.TrackSize)
{
if (this.Delta >= 0)
{
sb.Append('+');
}
sb.Append(this.Delta);
}
if (this.Address != 0)
{
sb.Append('@');
sb.Append(this.Address.ToString("x"));
}
//if (!string.IsNullOrEmpty(this.EventId))
//{
// sb.Append('@');
// sb.Append(this.EventId);
//}
}
return sb.ToString();
}
}
internal void Record(TrackedObjectKey key, CacheEvent evt, int? version, string eventId)
internal void Record(TrackedObjectKey key, CacheEvent evt, int? version, string eventId, long address)
{
Entry entry = new Entry
var info = this.GetObjectInfo(key);
info.CacheEvents.Enqueue(new Entry
{
EventId = eventId,
CacheEvent = evt,
Version = version,
};
Address = address,
});
this.Objects.AddOrUpdate(
key,
key => new ObjectInfo()
switch(evt)
{
case CacheEvent.StartingRMW:
case CacheEvent.PendingRMW:
info.PendingRMW = info.CurrentVersion;
break;
case CacheEvent.CompletedRMW:
if (info.CurrentVersion != info.PendingRMW + 1)
{
this.Fail("RMW completed without correctly updating the object", key);
}
info.PendingRMW = null;
break;
default:
break;
}
}
internal void UpdateTrackedObjectSize(long delta, TrackedObjectKey key, long? address)
{
var info = this.GetObjectInfo(key);
if (address == null)
{
info.GetCacheEvents(out _, out var entrySizes);
address = entrySizes.LastOrDefault().Key;
}
Interlocked.Add(ref info.Size, delta);
info.CacheEvents.Enqueue(new Entry
{
CacheEvent = CacheEvent.TrackSize,
Delta = delta,
Address = address.Value,
});
}
internal bool CheckSize(TrackedObjectKey key, List<(long delta, long address, string desc)> entries, long headAddress)
{
if (!this.EnableSizeChecking)
{
return false;
}
var info = this.GetObjectInfo(key);
long reference = Interlocked.Read(ref info.Size);
long actual = entries.Select(e => e.delta).Sum();
if (reference == actual)
{
info.CacheEvents.Enqueue(new Entry { CacheEvent = CacheEvent.SizeCheckSuccess, Delta = actual });
return true;
}
else
{
info.CacheEvents.Enqueue(new Entry { CacheEvent = CacheEvent.SizeCheckFail, Delta = actual, Address = reference });
// adjust the actual
var firstActual = entries.FirstOrDefault().address;
var firstReference = entries.FirstOrDefault().address;
var latestReadonly = info.CacheEvents.Where(e => e.CacheEvent == CacheEvent.Readonly).Select(e => e.Address).LastOrDefault();
var adjustedHead = Math.Max(Math.Max(firstActual, firstReference), latestReadonly + 1);
// try to account for concurrent eviction processing by using the latest head address
info.GetCacheEvents(out _, out var entrySizes);
var adjustedReference = entrySizes.Where(kvp => kvp.Key > adjustedHead).Select(kvp => kvp.Value).Sum();
var adjustedActual = entries.Where(e => e.address > adjustedHead).Select(e => e.delta).Sum();
// forcefully terminate if the adjusted size does not match
if (adjustedReference != adjustedActual)
{
CacheEvents = new List<Entry>() { entry },
},
(key, trace) =>
this.Fail($"Size tracking is not accurate reference={reference} actual={actual} referenceEntries={PrintExpectedEntries()} actualEntries={PrintActualEntries()} adjustedReference={adjustedReference} adjustedActual={adjustedActual} adjustedHead={adjustedHead:x} headAddress={headAddress:x}", key);
return false;
string PrintExpectedEntries()
{
var sb = new StringBuilder();
foreach (var kvp in entrySizes)
{
sb.Append($" {kvp.Value}@{kvp.Key:x}");
}
return sb.ToString();
}
string PrintActualEntries()
{
StringBuilder sb = new StringBuilder();
foreach (var e in entries)
{
sb.Append(' ');
sb.Append(e.desc);
}
return sb.ToString();
}
}
return false;
}
}
internal void UpdateSize(TrackedObjectKey key, long delta)
{
var info = this.GetObjectInfo(key);
Interlocked.Add(ref info.Size, delta);
}
internal void Reset(Func<string, bool> belongsToPartition)
{
// reset all size tracking for instances by this partition
foreach (var kvp in this.Objects)
{
if (belongsToPartition(kvp.Key.InstanceId))
{
trace.CacheEvents.Add(entry);
return trace;
});
var info = kvp.Value;
info.CacheEvents.Enqueue(new Entry() { CacheEvent = CacheEvent.Reset });
info.Size = 0;
}
}
}
internal void Fail(string message)
@ -146,10 +354,11 @@ namespace DurableTask.Netherite.Faster
internal void Fail(string message, TrackedObjectKey key)
{
this.Record(key, CacheEvent.Fail, null, null);
this.Record(key, CacheEvent.Fail, null, null, 0);
var objectInfo = this.Objects[key];
this.testHooks.Error(this.GetType().Name, $"{message} cacheEvents={objectInfo.PrintCacheEvents()}");
var info = this.GetObjectInfo(key);
this.testHooks.Error(nameof(CacheDebugger), $"{message} key={key} cacheEvents={info.PrintCacheEvents()}");
}
internal void ValidateObjectVersion(FasterKV.Value val, TrackedObjectKey key)
@ -178,65 +387,57 @@ namespace DurableTask.Netherite.Faster
if (val.Version != versionOfObject)
{
var objectInfo = this.Objects[key];
this.Fail($"incorrect version: reference=v{val.Version} actual=v{versionOfObject} obj={val.Val} cacheEvents={objectInfo.PrintCacheEvents()}");
var info = this.GetObjectInfo(key);
this.Fail($"incorrect version: model=v{val.Version} actual=v{versionOfObject} obj={val.Val} cacheEvents={info.PrintCacheEvents()}");
}
}
internal void CheckVersionConsistency(ref TrackedObjectKey key, TrackedObject obj, int? version)
internal void CheckVersionConsistency(TrackedObjectKey key, TrackedObject obj, int? version)
{
var objectInfo = this.Objects[key];
var info = this.GetObjectInfo(key);
if (version != null && version.Value != objectInfo.CurrentVersion)
if (version != null && version.Value != info.CurrentVersion)
{
this.Fail($"Read validation on version failed: expected=v{objectInfo.CurrentVersion} actual=v{version} obj={obj} cacheEvents={objectInfo.PrintCacheEvents()}");
this.Fail($"Read validation on version failed: reference=v{info.CurrentVersion} actual=v{version} obj={obj} key={key} cacheEvents={info.PrintCacheEvents()}");
}
if ((obj?.Version ?? 0) != objectInfo.CurrentVersion)
if ((obj?.Version ?? 0) != info.CurrentVersion)
{
this.Fail($"Read validation on object failed: expected=v{objectInfo.CurrentVersion} actual=v{obj?.Version ?? 0} obj={obj} cacheEvents={objectInfo.PrintCacheEvents()}");
this.Fail($"Read validation on object failed: reference=v{info.CurrentVersion} actual=v{obj?.Version ?? 0} obj={obj} key={key} cacheEvents={info.PrintCacheEvents()}");
}
}
internal void CheckIteratorAbsence(TrackedObjectKey key)
{
var info = this.GetObjectInfo(key);
if (info.CurrentVersion != 0)
{
this.Fail($"scan iterator missed object v{info.CurrentVersion} key={key} cacheEvents={info.PrintCacheEvents()}");
}
}
internal void UpdateReferenceValue(ref TrackedObjectKey key, TrackedObject obj, int version)
{
this.Objects.AddOrUpdate(
key,
key => new ObjectInfo()
{
CurrentVersion = version,
CacheEvents = new List<Entry>(),
},
(key, trace) =>
{
trace.CurrentVersion = version;
return trace;
});
if (obj != null)
{
obj.Version = version;
}
}
internal void UpdateReferenceValue(ref TrackedObjectKey key, object obj, int version)
{
if (obj is byte[] bytes)
{
this.UpdateReferenceValue(ref key, (TrackedObject) DurableTask.Netherite.Serializer.DeserializeTrackedObject(bytes), version);
}
else
{
this.UpdateReferenceValue(ref key, obj as TrackedObject, version);
}
var info = this.GetObjectInfo(key);
info.CurrentVersion = version;
}
public IEnumerable<string> Dump()
{
this.SizeCheckFailed = false;
foreach (var kvp in this.Objects)
{
yield return $"{kvp.Key,-25} {string.Join(",", kvp.Value.CacheEvents.Select(e => e.ToString()))}";
}
string eventList = string.Join(",", kvp.Value.CacheEvents.Select(e => e.ToString()));
var lastSizeCheck = kvp.Value.CacheEvents.LastOrDefault(e => e.CacheEvent == CacheEvent.SizeCheckFail || e.CacheEvent == CacheEvent.SizeCheckSuccess);
bool fail = lastSizeCheck.CacheEvent == CacheEvent.SizeCheckFail;
this.SizeCheckFailed = this.SizeCheckFailed || fail;
yield return $"{kvp.Key,-25} {eventList} {(fail ? "SIZECHECKFAIL" : "")}";
}
}
public bool SizeCheckFailed { get; set; }
}
}

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

@ -25,6 +25,8 @@ namespace DurableTask.Netherite.Faster
CheckpointDueAsync handler;
TaskCompletionSource<LogAccessor<FasterKV.Key, FasterKV.Value>> continuation;
public bool InjectFaultAfterCompaction { get; private set; }
public CheckpointInjector(TestHooks testHooks)
{
this.testHooks = testHooks;
@ -62,14 +64,26 @@ namespace DurableTask.Netherite.Faster
internal void SequenceComplete(LogAccessor<FasterKV.Key, FasterKV.Value> log)
{
this.continuation.SetResult(log);
this.continuation?.SetResult(log);
this.continuation = null;
}
internal Task<LogAccessor<FasterKV.Key, FasterKV.Value>> InjectAsync(CheckpointDueAsync handler)
internal void CompactionComplete(IPartitionErrorHandler errorHandler)
{
if (this.InjectFaultAfterCompaction)
{
errorHandler.HandleError("CheckpointInjector", "inject failure after compaction", null, true, false);
this.InjectFaultAfterCompaction = false;
this.continuation?.SetResult(null);
this.continuation = null;
}
}
internal Task<LogAccessor<FasterKV.Key, FasterKV.Value>> InjectAsync(CheckpointDueAsync handler, bool injectFailureAfterCompaction = false)
{
this.continuation = new TaskCompletionSource<LogAccessor<FasterKV.Key, FasterKV.Value>>(TaskCreationOptions.RunContinuationsAsynchronously);
this.handler = handler;
this.InjectFaultAfterCompaction = injectFailureAfterCompaction;
return this.continuation.Task;
}
}

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

@ -88,10 +88,16 @@ namespace DurableTask.Netherite.Faster
this.blobManager.TraceHelper.FasterProgress("Constructed FasterAlt");
}
public override (double totalSizeMB, int fillPercentage) CacheSizeInfo => (0.0,0);
public override void InitMainSession()
{
}
public override void AdjustCacheSize()
{
}
public override Task<(long commitLogPosition, long inputQueuePosition)> RecoverAsync()
{
foreach (var guid in this.ReadCheckpointIntentions())
@ -122,7 +128,7 @@ namespace DurableTask.Netherite.Faster
return Task.FromResult(dedupState.Positions);
}
public override void CompletePending()
public override bool CompletePending()
{
var completed = this.pendingLoads.Where(p => p.Value.LoadTask.IsCompleted).ToList();
@ -130,6 +136,8 @@ namespace DurableTask.Netherite.Faster
{
this.ProcessCompletedLoad(kvp.Key, kvp.Value);
}
return this.pendingLoads.Count == 0;
}
public override ValueTask ReadyToCompletePendingAsync()
@ -227,6 +235,11 @@ namespace DurableTask.Netherite.Faster
return this.RemoveCheckpointIntention(guid);
}
public override void CheckInvariants()
{
}
// perform a query
public override Task QueryAsync(PartitionQueryEvent queryEvent, EffectTracker effectTracker)
{
@ -235,7 +248,7 @@ namespace DurableTask.Netherite.Faster
}
// kick off a read of a tracked object, completing asynchronously if necessary
public override void ReadAsync(PartitionReadEvent readEvent, EffectTracker effectTracker)
public override void Read(PartitionReadEvent readEvent, EffectTracker effectTracker)
{
if (readEvent.Prefetch.HasValue)
{
@ -403,7 +416,7 @@ namespace DurableTask.Netherite.Faster
async Task<ToRead> LoadAsync(TrackedObjectKey key)
{
this.detailTracer?.FasterStorageProgress($"FasterAlt.LoadAsync Called key={key}");
this.detailTracer?.FasterStorageProgress($"StorageOpCalled FasterAlt.LoadAsync key={key}");
try
{
@ -434,7 +447,7 @@ namespace DurableTask.Netherite.Faster
toRead.NewValue = newLength > 0 ? reader.ReadBytes(newLength) : null;
toRead.Guid = new Guid(reader.ReadBytes(16));
this.detailTracer?.FasterStorageProgress($"FasterAlt.LoadAsync Returned key={key}");
this.detailTracer?.FasterStorageProgress($"StorageOpReturned FasterAlt.LoadAsync key={key}");
return toRead;
}
catch (StorageException) when (this.terminationToken.IsCancellationRequested)
@ -443,10 +456,10 @@ namespace DurableTask.Netherite.Faster
}
catch (StorageException ex) when (BlobUtils.BlobDoesNotExist(ex))
{
this.detailTracer?.FasterStorageProgress($"FasterAlt.LoadAsync Returned 404 key={key}");
this.detailTracer?.FasterStorageProgress($"StorageOpReturned FasterAlt.LoadAsync got 404 key={key}");
return default;
}
catch (StorageException e) when (BlobUtils.IsTransientStorageError(e, this.terminationToken) && numAttempts < BlobManager.MaxRetries)
catch (Exception e) when (BlobUtils.IsTransientStorageError(e, this.terminationToken) && numAttempts < BlobManager.MaxRetries)
{
TimeSpan nextRetryIn = BlobManager.GetDelayBetweenRetries(numAttempts);
this.blobManager?.HandleStorageError(nameof(LoadAsync), $"Could not read object from storage, will retry in {nextRetryIn}s, numAttempts={numAttempts}", blob.Name, e, false, true);
@ -468,7 +481,7 @@ namespace DurableTask.Netherite.Faster
async Task StoreAsync(byte[] guid, ToWrite entry)
{
this.detailTracer?.FasterStorageProgress($"FasterAlt.LoadAsync Called {entry.Key}");
this.detailTracer?.FasterStorageProgress($"StorageOpCalled FasterAlt.LoadAsync {entry.Key}");
// assemble the bytes to write
using var stream = new MemoryStream();
@ -521,7 +534,7 @@ namespace DurableTask.Netherite.Faster
{
throw new OperationCanceledException("Partition was terminated.", this.terminationToken);
}
catch (StorageException e) when (BlobUtils.IsTransientStorageError(e, this.blobManager.PartitionErrorHandler.Token) && numAttempts < BlobManager.MaxRetries)
catch (Exception e) when (BlobUtils.IsTransientStorageError(e, this.blobManager.PartitionErrorHandler.Token) && numAttempts < BlobManager.MaxRetries)
{
TimeSpan nextRetryIn = BlobManager.GetDelayBetweenRetries(numAttempts);
this.blobManager?.HandleStorageError(nameof(StoreAsync), $"could not write object to storage, will retry in {nextRetryIn}s, numAttempts={numAttempts}", blob.Name, e, false, true);

Разница между файлами не показана из-за своего большого размера Загрузить разницу

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

@ -11,29 +11,33 @@ namespace DurableTask.Netherite.Faster
class FasterLog
{
readonly BlobManager blobManager;
readonly FASTER.core.FasterLog log;
readonly CancellationToken terminationToken;
public FasterLog(BlobManager blobManager, NetheriteOrchestrationServiceSettings settings)
{
this.blobManager = blobManager;
var eventlogsettings = blobManager.GetDefaultEventLogSettings(settings.UseSeparatePageBlobStorage, settings.FasterTuningParameters);
this.log = new FASTER.core.FasterLog(eventlogsettings);
blobManager.PartitionErrorHandler.OnShutdown += this.Shutdown;
this.terminationToken = blobManager.PartitionErrorHandler.Token;
}
void Shutdown()
{
try
{
this.blobManager.TraceHelper.FasterProgress("Disposing FasterLog");
this.log.Dispose();
var _ = this.terminationToken.Register(
() => {
try
{
this.log.Dispose();
blobManager.EventLogDevice.Dispose();
}
catch (Exception e)
{
blobManager.TraceHelper.FasterStorageError("Disposing FasterLog", e);
}
},
useSynchronizationContext: false);
this.blobManager.TraceHelper.FasterProgress("Disposing FasterLog Device");
this.blobManager.EventLogDevice.Dispose();
}
catch (Exception e)
{
this.blobManager.TraceHelper.FasterStorageError("Disposing FasterLog", e);
}
}
public long BeginAddress => this.log.BeginAddress;

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

@ -20,6 +20,7 @@ namespace DurableTask.Netherite.Faster
readonly string taskHubName;
readonly string pathPrefix;
readonly ILogger logger;
readonly MemoryTracker memoryTracker;
Partition partition;
BlobManager blobManager;
@ -33,11 +34,16 @@ namespace DurableTask.Netherite.Faster
internal FasterTraceHelper TraceHelper { get; private set; }
public FasterStorage(string connectionString, string pageBlobConnectionString, string localFileDirectory, string taskHubName, string pathPrefix, ILoggerFactory loggerFactory)
public long TargetMemorySize { get; set; }
public FasterStorage(NetheriteOrchestrationServiceSettings settings, string pathPrefix, MemoryTracker memoryTracker, ILoggerFactory loggerFactory)
{
if (!string.IsNullOrEmpty(localFileDirectory))
string connectionString = settings.ResolvedStorageConnectionString;
string pageBlobConnectionString = settings.ResolvedPageBlobStorageConnectionString;
if (!string.IsNullOrEmpty(settings.UseLocalDirectoryForPartitionStorage))
{
this.localFileDirectory = localFileDirectory;
this.localFileDirectory = settings.UseLocalDirectoryForPartitionStorage;
}
else
{
@ -51,9 +57,15 @@ namespace DurableTask.Netherite.Faster
{
this.pageBlobStorageAccount = this.storageAccount;
}
this.taskHubName = taskHubName;
this.taskHubName = settings.HubName;
this.pathPrefix = pathPrefix;
this.logger = loggerFactory.CreateLogger($"{NetheriteOrchestrationService.LoggerCategoryName}.FasterStorage");
this.memoryTracker = memoryTracker;
if (settings.TestHooks?.CacheDebugger != null)
{
settings.TestHooks.CacheDebugger.MemoryTracker = this.memoryTracker;
}
}
public static Task DeleteTaskhubStorageAsync(string connectionString, string pageBlobConnectionString, string localFileDirectory, string taskHubName, string pathPrefix)
@ -121,7 +133,7 @@ namespace DurableTask.Netherite.Faster
else
{
this.TraceHelper.FasterProgress("Creating FasterKV");
this.store = new FasterKV(this.partition, this.blobManager);
this.store = new FasterKV(this.partition, this.blobManager, this.memoryTracker);
}
this.TraceHelper.FasterProgress("Creating StoreWorker");

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

@ -142,6 +142,16 @@ namespace DurableTask.Netherite.Faster
}
}
public enum CompactionProgress { Skipped, Started, Completed };
public void FasterCompactionProgress(CompactionProgress progress, string operation, long begin, long safeReadOnly, long tail, long minimalSize, long compactionAreaSize, long elapsedMs)
{
if (this.logLevelLimit <= LogLevel.Information)
{
this.logger.LogInformation("Part{partition:D2} Compaction {progress} operation={operation} begin={begin} safeReadOnly={safeReadOnly} tail={tail} minimalSize={minimalSize} compactionAreaSize={compactionAreaSize} elapsedMs={elapsedMs}", this.partitionId, progress, operation, begin, safeReadOnly, tail, minimalSize, compactionAreaSize, elapsedMs);
this.etwLogInformation?.FasterCompactionProgress(this.account, this.taskHub, this.partitionId, progress.ToString(), operation, begin, safeReadOnly, tail, minimalSize, compactionAreaSize, elapsedMs, TraceUtils.AppName, TraceUtils.ExtensionVersion);
}
}
// ----- lease management events

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

@ -32,6 +32,8 @@ namespace DurableTask.Netherite.Faster
public int RandomProbability { get; set; }
readonly Random random = new Random();
int failedRestarts = 1;
public void StartNewTest()
{
System.Diagnostics.Trace.TraceInformation($"FaultInjector: StartNewTest");
@ -143,12 +145,20 @@ namespace DurableTask.Netherite.Faster
if (this.RandomProbability > 0)
{
var dieRoll = this.random.Next(this.RandomProbability);
//Console.WriteLine(dieRoll);
if (dieRoll == 0)
if (this.failedRestarts > 0 && this.startedPartitions.Contains(blobManager))
{
pass = false;
this.failedRestarts = 0;
}
if (this.failedRestarts < 2)
{
var dieRoll = this.random.Next(this.RandomProbability * (1 + this.failedRestarts));
if (dieRoll == 0)
{
pass = false;
this.failedRestarts++;
}
}
}

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

@ -35,7 +35,7 @@ namespace DurableTask.Netherite.Faster
this.traceHelper = traceHelper;
this.intakeWorker = new IntakeWorker(cancellationToken, this, partition.TraceHelper);
this.maxFragmentSize = (1 << this.blobManager.GetDefaultEventLogSettings(partition.Settings.UseSeparatePageBlobStorage, partition.Settings.FasterTuningParameters).PageSizeBits) - 64; // faster needs some room for header, 64 bytes is conservative
this.maxFragmentSize = (int) this.blobManager.GetDefaultEventLogSettings(partition.Settings.UseSeparatePageBlobStorage, partition.Settings.FasterTuningParameters).PageSize - 64; // faster needs some room for header, 64 bytes is conservative
}
public const byte first = 0x1;
@ -133,9 +133,9 @@ namespace DurableTask.Netherite.Faster
this.isShuttingDown = true;
await this.intakeWorker.WaitForCompletionAsync().ConfigureAwait(false);
await this.intakeWorker.WaitForCompletionAsync();
await this.WaitForCompletionAsync().ConfigureAwait(false);
await this.WaitForCompletionAsync();
this.traceHelper.FasterProgress($"Stopped LogWorker");
}
@ -151,7 +151,7 @@ namespace DurableTask.Netherite.Faster
stopwatch.Start();
long previous = this.log.CommittedUntilAddress;
await this.log.CommitAsync().ConfigureAwait(false); // may commit more events than just the ones in the batch, but that is o.k.
await this.log.CommitAsync(); // may commit more events than just the ones in the batch, but that is o.k.
this.traceHelper.FasterLogPersisted(this.log.CommittedUntilAddress, batch.Count, (this.log.CommittedUntilAddress - previous), stopwatch.ElapsedMilliseconds);
@ -259,7 +259,7 @@ namespace DurableTask.Netherite.Faster
{
yield break;
}
await iter.WaitAsync(this.cancellationToken).ConfigureAwait(false);
await iter.WaitAsync(this.cancellationToken);
}
if ((result[0] & first) != none)

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

@ -0,0 +1,175 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
namespace DurableTask.Netherite.Faster
{
using System;
using System.Collections.Concurrent;
using System.Collections.Generic;
using System.Diagnostics;
using System.Linq;
using System.Text;
using System.Threading;
using System.Threading.Tasks;
using FASTER.core;
/// <summary>
/// Track memory use by all FASTER caches.
/// </summary>
class MemoryTracker
{
readonly long maxCacheSize;
readonly ConcurrentDictionary<CacheTracker, CacheTracker> stores;
public long MaxCacheSize => this.maxCacheSize;
public const int MinimumMemoryPages = 1;
public MemoryTracker(long maxCacheSize)
{
this.maxCacheSize = maxCacheSize;
this.stores = new ConcurrentDictionary<CacheTracker, CacheTracker>();
}
public CacheTracker NewCacheTracker(FasterKV store, int partitionId, CacheDebugger cacheDebugger)
{
var cacheTracker = new CacheTracker(this, partitionId, store, cacheDebugger);
this.stores.TryAdd(cacheTracker, cacheTracker);
this.UpdateTargetSizes();
return cacheTracker;
}
public void UpdateTargetSizes()
{
if (this.stores.Count > 0)
{
long targetSize = this.maxCacheSize / this.stores.Count;
foreach (var s in this.stores.Keys)
{
s.SetTargetSize(targetSize);
}
}
}
internal (int, long) GetMemorySize() // used for testing only
{
(int totalPages, long totalSize) = (0, 0);
foreach(var store in this.stores.Values)
{
(int numPages, long size) = store.ComputeMemorySize();
totalPages += numPages;
totalSize += size;
}
return (totalPages, totalSize);
}
internal void SetEmptyPageCount(int emptyPageCount) // used by tests only
{
foreach (var store in this.stores.Values)
{
store.SetEmptyPageCount(emptyPageCount);
}
}
public class CacheTracker : BatchWorker<object>, IDisposable
{
readonly MemoryTracker memoryTracker;
readonly FasterKV store;
readonly CacheDebugger cacheDebugger;
readonly int pageSizeBits;
long trackedObjectSize;
public long TrackedObjectSize => Interlocked.Read(ref this.trackedObjectSize);
public long TargetSize { get; set; }
public long MaxCacheSize => this.memoryTracker.maxCacheSize;
public CacheTracker(MemoryTracker memoryTracker, int partitionId, FasterKV store, CacheDebugger cacheDebugger)
: base($"CacheTracker{partitionId:D2}", false, 10000, CancellationToken.None, null)
{
this.memoryTracker = memoryTracker;
this.store = store;
this.cacheDebugger = cacheDebugger;
this.pageSizeBits = store.PageSizeBits;
}
public void Dispose()
{
if (this.memoryTracker.stores.TryRemove(this, out _))
{
this.memoryTracker.UpdateTargetSizes();
}
}
public void MeasureCacheSize()
{
Stopwatch stopwatch = new Stopwatch();
stopwatch.Start();
(int numPages, long size) = this.store.ComputeMemorySize(true);
double MB(long bytes) => (double)bytes / (1024 * 1024);
this.store.TraceHelper.FasterProgress($"CacheSize: numPages={numPages} objectSize={MB(size):F2}MB totalSize={MB(size + this.store.MemoryUsedWithoutObjects):F2}MB elapsedMs={stopwatch.Elapsed.TotalMilliseconds:F2}");
this.trackedObjectSize = size;
}
public (int, long) ComputeMemorySize() => this.store.ComputeMemorySize(false); // used by tests only
internal void SetEmptyPageCount(int emptyPageCount) => this.store.SetEmptyPageCount(emptyPageCount); // used by tests only
public void SetTargetSize(long newTargetSize)
{
this.TargetSize = newTargetSize;
this.Notify();
}
public void OnEviction(long totalSize, long endAddress)
{
Interlocked.Add(ref this.trackedObjectSize, -totalSize);
this.Notify();
}
internal void UpdateTrackedObjectSize(long delta, TrackedObjectKey key, long? address)
{
Interlocked.Add(ref this.trackedObjectSize, delta);
this.cacheDebugger?.UpdateTrackedObjectSize(delta, key, address);
}
protected override Task Process(IList<object> _)
{
var log = this.store.Log;
if (log != null)
{
long excess = Interlocked.Read(ref this.trackedObjectSize) + this.store.MemoryUsedWithoutObjects - this.TargetSize;
long firstPage = this.store.Log.HeadAddress >> this.pageSizeBits;
long lastPage = this.store.Log.TailAddress >> this.pageSizeBits;
int numUsedPages = (int) ((lastPage - firstPage) + 1);
int actuallyEmptyPages = log.BufferSize - numUsedPages;
int currentTarget = Math.Max(log.EmptyPageCount, actuallyEmptyPages);
int tighten = Math.Min(currentTarget + 1, log.BufferSize - MinimumMemoryPages);
int loosen = 0;
if (excess > 0 && currentTarget < tighten)
{
this.store.TraceHelper.FasterStorageProgress($"MemoryControl Engage tighten={tighten} EmptyPageCount={log.EmptyPageCount} excess={excess / 1024}kB actuallyEmptyPages={actuallyEmptyPages}");
log.SetEmptyPageCount(tighten, true);
this.Notify();
}
else if (excess < 0 && log.EmptyPageCount > loosen)
{
this.store.TraceHelper.FasterStorageProgress($"MemoryControl Disengage loosen={loosen} EmptyPageCount={log.EmptyPageCount} excess={excess / 1024}kB actuallyEmptyPages={actuallyEmptyPages}");
log.SetEmptyPageCount(loosen, true);
this.Notify();
}
else
{
this.store.TraceHelper.FasterStorageProgress($"MemoryControl Steady EmptyPageCount={log.EmptyPageCount} excess={excess / 1024}kB tighten={tighten} loosen={loosen} actuallyEmptyPages={actuallyEmptyPages}");
}
}
return Task.CompletedTask;
}
}
}
}

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

@ -154,8 +154,10 @@ namespace DurableTask.Netherite.Faster
internal async Task CheckUpdate(Partition partition, PartitionUpdateEvent partitionUpdateEvent, TrackedObjectStore store)
{
var info = this.partitionInfo[partition];
var cacheDebugger = partition.Settings.TestHooks?.CacheDebugger;
System.Diagnostics.Trace.WriteLine($"REPLAYCHECK STARTED {partitionUpdateEvent}");
partition.EventTraceHelper.TraceEventProcessingDetail($"REPLAYCHECK-STARTED {partitionUpdateEvent.NextCommitLogPosition}");
int errors = 0;
string serializedEvent = this.Serialize(partitionUpdateEvent);
var eventForReplay = this.DeserializePartitionUpdateEvent(serializedEvent);
@ -169,45 +171,62 @@ namespace DurableTask.Netherite.Faster
store.EmitCurrentState((TrackedObjectKey key, TrackedObject value) =>
{
NotVisited.Remove(key);
string expected = this.Serialize(value);
string current = this.Serialize(value);
if (!info.Store.TryGetValue(key, out var replayed))
{
this.testHooks.Error(this.GetType().Name, $"key={key}\nexpected={expected}\nreplayed=absent");
info.Store[key] = expected;
this.testHooks.Error(this.GetType().Name, $"Part{partition.PartitionId:D2} pos={partitionUpdateEvent.NextCommitLogPosition} key={key}"
+ $"\ncurrent={current}\nreplayed=absent\nevent={serializedEvent}");
info.Store[key] = current;
}
if (expected != replayed)
if (current != replayed)
{
var expectedlines = TraceUtils.GetLines(expected).ToArray();
var currentlines = TraceUtils.GetLines(current).ToArray();
var replayedlines = TraceUtils.GetLines(replayed).ToArray();
string expectedline = "";
string currentline = "";
string replayedline = "";
int i = 0;
for (; i < Math.Max(expectedlines.Length, replayedlines.Length); i++)
for (; i < Math.Max(currentlines.Length, replayedlines.Length); i++)
{
expectedline = i < expectedlines.Length ? expectedlines[i] : "absent";
currentline = i < currentlines.Length ? currentlines[i] : "absent";
replayedline = i < replayedlines.Length ? replayedlines[i] : "absent";
if (expectedline != replayedline)
if (currentline != replayedline)
{
break;
}
}
this.testHooks.Error(this.GetType().Name, $"Part{partition.PartitionId:D2} pos={partitionUpdateEvent.NextCommitLogPosition} key={key}"
+ $"\nline={i}\nexpectedline={expectedline}\nreplayedline={replayedline}\nexpected={expected}\nreplayed={replayed}\nevent={serializedEvent}");
info.Store[key] = expected;
+ $"\nline={i}\ncurrentline={currentline}\nreplayedline={replayedline}\ncurrent={current}\nreplayed={replayed}\nevent={serializedEvent}");
info.Store[key] = current;
errors++;
}
});
foreach(var key in NotVisited)
foreach (var key in NotVisited)
{
if (!key.IsSingleton)
{
cacheDebugger?.CheckIteratorAbsence(key);
}
string val = info.Store[key];
this.testHooks.Error(this.GetType().Name, $"key={key}\nexpected=absent\nreplayed={val}");
this.testHooks.Error(this.GetType().Name, $"Part{partition.PartitionId:D2} pos={partitionUpdateEvent.NextCommitLogPosition} key={key}"
+ $"\ncurrent=absent\nreplayed={val}\nevent={serializedEvent}");
info.Store.Remove(key);
errors++;
}
System.Diagnostics.Trace.WriteLine("REPLAYCHECK DONE");
if (errors == 0)
{
partition.EventTraceHelper.TraceEventProcessingDetail($"REPLAYCHECK-SUCCEEDED {partitionUpdateEvent.NextCommitLogPosition}");
}
else
{
partition.EventTraceHelper.TraceEventProcessingDetail($"REPLAYCHECK-FAILED {partitionUpdateEvent.NextCommitLogPosition} ({errors} ERRORS)");
}
}
internal void PartitionStopped(Partition partition)
{
this.partitionInfo.TryRemove(partition, out _);

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

@ -34,11 +34,10 @@ namespace DurableTask.Netherite.Faster
long lastCheckpointedInputQueuePosition;
long lastCheckpointedCommitLogPosition;
long numberEventsSinceLastCheckpoint;
long timeOfNextIdleCheckpoint;
DateTime timeOfNextIdleCheckpoint;
// periodic compaction
Task<long?> pendingCompaction;
long? shiftPending;
// periodic load publishing
PartitionLoadInfo loadInfo;
@ -79,7 +78,7 @@ namespace DurableTask.Netherite.Faster
foreach (var key in TrackedObjectKey.GetSingletons())
{
var target = await this.store.CreateAsync(key).ConfigureAwait(false);
var target = await this.store.CreateAsync(key);
target.OnFirstInitialization();
}
@ -117,8 +116,8 @@ namespace DurableTask.Netherite.Faster
this.traceHelper.FasterCheckpointStarted(checkpointGuid, reason, this.store.StoreStats.Get(), this.CommitLogPosition, this.InputQueuePosition);
// do the faster full checkpoint and then finalize it
await this.store.CompleteCheckpointAsync().ConfigureAwait(false);
await this.store.FinalizeCheckpointCompletedAsync(checkpointGuid).ConfigureAwait(false);
await this.store.CompleteCheckpointAsync();
await this.store.FinalizeCheckpointCompletedAsync(checkpointGuid);
this.lastCheckpointedCommitLogPosition = this.CommitLogPosition;
this.lastCheckpointedInputQueuePosition = this.InputQueuePosition;
@ -140,16 +139,16 @@ namespace DurableTask.Netherite.Faster
this.isShuttingDown = true;
await this.WaitForCompletionAsync().ConfigureAwait(false);
await this.WaitForCompletionAsync();
// wait for any in-flight checkpoints. It is unlikely but not impossible.
if (this.pendingIndexCheckpoint != null)
{
await this.pendingIndexCheckpoint.ConfigureAwait(false);
await this.pendingIndexCheckpoint;
}
if (this.pendingStoreCheckpoint != null)
{
await this.pendingStoreCheckpoint.ConfigureAwait(false);
await this.pendingStoreCheckpoint;
}
this.replayChecker?.PartitionStopped(this.partition);
@ -166,10 +165,11 @@ namespace DurableTask.Netherite.Faster
{
foreach (var k in TrackedObjectKey.GetSingletons())
{
(await this.store.ReadAsync(k, this.effectTracker).ConfigureAwait(false)).UpdateLoadInfo(this.loadInfo);
(await this.store.ReadAsync(k, this.effectTracker)).UpdateLoadInfo(this.loadInfo);
}
this.loadInfo.MissRate = this.store.StoreStats.GetMissRate();
(this.loadInfo.CacheMB, this.loadInfo.CachePct) = this.store.CacheSizeInfo;
if (this.loadInfo.IsBusy() != null)
{
@ -222,7 +222,6 @@ namespace DurableTask.Netherite.Faster
bool CheckpointDue(out CheckpointTrigger trigger, out long? compactUntil)
{
// in a test setting, let the test decide when to checkpoint or compact
if (this.partition.Settings.TestHooks?.CheckpointInjector != null)
{
@ -244,9 +243,10 @@ namespace DurableTask.Netherite.Faster
{
trigger = CheckpointTrigger.EventCount;
}
else if (this.loadInfo.IsBusy() == null && DateTime.UtcNow.Ticks > this.timeOfNextIdleCheckpoint)
else if (this.loadInfo.IsBusy() == null && DateTime.UtcNow > this.timeOfNextIdleCheckpoint)
{
// we have reached an idle point.
this.ScheduleNextIdleCheckpointTime();
compactUntil = this.store.GetCompactionTarget();
if (compactUntil.HasValue)
@ -281,11 +281,12 @@ namespace DurableTask.Netherite.Faster
void ScheduleNextIdleCheckpointTime()
{
// to avoid all partitions taking snapshots at the same time, align to a partition-based spot between 0.5 and 1.5 of the period
var period = this.partition.Settings.IdleCheckpointFrequencyMs * TimeSpan.TicksPerMillisecond;
// to avoid all partitions taking snapshots at the same time, align to a partition-based spot between 0.5 and 1.5 of the period
var earliest = DateTime.UtcNow.Ticks + period/2;
var offset = (this.partition.PartitionId * period / this.partition.NumberPartitions());
this.timeOfNextIdleCheckpoint = earliest + offset;
var offset = this.partition.PartitionId * period / this.partition.NumberPartitions();
var earliest = DateTime.UtcNow.Ticks + period / 2;
var actual = (((earliest - offset - 1) / period) + 1) * period + offset;
this.timeOfNextIdleCheckpoint = new DateTime(actual, DateTimeKind.Utc);
}
protected override async Task Process(IList<PartitionEvent> batch)
@ -309,12 +310,12 @@ namespace DurableTask.Netherite.Faster
switch (partitionEvent)
{
case PartitionUpdateEvent updateEvent:
await this.ProcessUpdate(updateEvent).ConfigureAwait(false);
await this.ProcessUpdate(updateEvent);
break;
case PartitionReadEvent readEvent:
// async reads may either complete immediately (on cache hit) or later (on cache miss) when CompletePending() is called
this.store.ReadAsync(readEvent, this.effectTracker);
this.store.Read(readEvent, this.effectTracker);
break;
default:
@ -339,7 +340,9 @@ namespace DurableTask.Netherite.Faster
return;
}
// handle progression of checkpointing state machine: none -> pendingCompaction -> pendingIndexCheckpoint -> { pendingStoreCheckpoint (shiftpending.HasValue) -> } pendingStoreCheckpoint (shiftpending == null) -> none)
this.store.AdjustCacheSize();
// handle progression of checkpointing state machine: none -> pendingCompaction -> pendingIndexCheckpoint -> pendingStoreCheckpoint -> none)
if (this.pendingStoreCheckpoint != null)
{
if (this.pendingStoreCheckpoint.IsCompleted == true)
@ -347,25 +350,11 @@ namespace DurableTask.Netherite.Faster
(this.lastCheckpointedCommitLogPosition, this.lastCheckpointedInputQueuePosition)
= await this.pendingStoreCheckpoint; // observe exceptions here
if (this.shiftPending.HasValue)
{
// shifting, and then a second store checkpoint, are next
var token = this.store.StartStoreCheckpoint(this.CommitLogPosition, this.InputQueuePosition, this.shiftPending);
if (token.HasValue)
{
this.shiftPending = null;
this.pendingStoreCheckpoint = this.WaitForCheckpointAsync(false, token.Value, true);
this.numberEventsSinceLastCheckpoint = 0;
}
}
else
{
// we have reached the end of the state machine transitions
this.pendingStoreCheckpoint = null;
this.pendingCheckpointTrigger = CheckpointTrigger.None;
this.ScheduleNextIdleCheckpointTime();
this.partition.Settings.TestHooks?.CheckpointInjector?.SequenceComplete((this.store as FasterKV).Log);
}
// we have reached the end of the state machine transitions
this.pendingStoreCheckpoint = null;
this.pendingCheckpointTrigger = CheckpointTrigger.None;
this.ScheduleNextIdleCheckpointTime();
this.partition.Settings.TestHooks?.CheckpointInjector?.SequenceComplete((this.store as FasterKV).Log);
}
}
else if (this.pendingIndexCheckpoint != null)
@ -374,12 +363,12 @@ namespace DurableTask.Netherite.Faster
{
await this.pendingIndexCheckpoint; // observe exceptions here
// the (first) store checkpoint is next
// the store checkpoint is next
var token = this.store.StartStoreCheckpoint(this.CommitLogPosition, this.InputQueuePosition, null);
if (token.HasValue)
{
this.pendingIndexCheckpoint = null;
this.pendingStoreCheckpoint = this.WaitForCheckpointAsync(false, token.Value, !this.shiftPending.HasValue);
this.pendingStoreCheckpoint = this.WaitForCheckpointAsync(false, token.Value, true);
this.numberEventsSinceLastCheckpoint = 0;
}
}
@ -388,8 +377,7 @@ namespace DurableTask.Netherite.Faster
{
if (this.pendingCompaction.IsCompleted == true)
{
// take not of the shifted begin address, we will use it when it comes time for the store checkpoint
this.shiftPending = await this.pendingCompaction;
await this.pendingCompaction; // observe exceptions here
// the index checkpoint is next
var token = this.store.StartIndexCheckpoint();
@ -410,33 +398,31 @@ namespace DurableTask.Netherite.Faster
// periodically publish the partition load information
if (this.lastPublished + PublishInterval < DateTime.UtcNow)
{
await this.PublishPartitionLoad().ConfigureAwait(false);
await this.PublishPartitionLoad();
}
if (ActivityScheduling.RequiresLoadMonitor(this.partition.Settings.ActivityScheduler))
{
var activitiesState = (await this.store.ReadAsync(TrackedObjectKey.Activities, this.effectTracker).ConfigureAwait(false)) as ActivitiesState;
var activitiesState = (await this.store.ReadAsync(TrackedObjectKey.Activities, this.effectTracker)) as ActivitiesState;
activitiesState.CollectLoadMonitorInformation();
}
if (this.loadInfo.IsBusy() != null
|| (this.lastCheckpointedCommitLogPosition == this.CommitLogPosition
&& this.lastCheckpointedInputQueuePosition == this.InputQueuePosition
&& this.LogWorker.LastCommittedInputQueuePosition <= this.InputQueuePosition))
if (this.loadInfo.IsBusy() != null)
{
// the partition is not idle, or nothing has changed, so we do delay the time for the nex idle checkpoint
// the partition is not idle, so we do delay the time for the next idle checkpoint
this.ScheduleNextIdleCheckpointTime();
}
// make sure to complete ready read requests, or notify this worker
// if any read requests become ready to process at some point
var t = this.store.ReadyToCompletePendingAsync();
if (!t.IsCompleted)
// we always call this at the end of the loop.
bool allRequestsCompleted = this.store.CompletePending();
if (!allRequestsCompleted)
{
var ignoredTask = t.AsTask().ContinueWith(x => this.Notify());
var _ = this.store.ReadyToCompletePendingAsync().AsTask().ContinueWith(x => this.Notify());
}
// we always call this at the end of the loop.
this.store.CompletePending();
// during testing, this is a good time to check invariants in the store
this.store.CheckInvariants();
}
catch (OperationCanceledException) when (this.cancellationToken.IsCancellationRequested)
{
@ -458,15 +444,15 @@ namespace DurableTask.Netherite.Faster
this.traceHelper.FasterCheckpointStarted(checkpointToken, description, this.store.StoreStats.Get(), commitLogPosition, inputQueuePosition);
// first do the faster checkpoint
await this.store.CompleteCheckpointAsync().ConfigureAwait(false);
await this.store.CompleteCheckpointAsync();
if (!isIndexCheckpoint)
{
// wait for the commit log so it is never behind the checkpoint
await this.LogWorker.WaitForCompletionAsync().ConfigureAwait(false);
await this.LogWorker.WaitForCompletionAsync();
// finally we write the checkpoint info file
await this.store.FinalizeCheckpointCompletedAsync(checkpointToken).ConfigureAwait(false);
await this.store.FinalizeCheckpointCompletedAsync(checkpointToken);
// notify the log worker that the log can be truncated up to the commit log position
this.LogWorker.SetLastCheckpointPosition(commitLogPosition);
@ -488,6 +474,8 @@ namespace DurableTask.Netherite.Faster
if (target.HasValue)
{
target = await this.store.RunCompactionAsync(target.Value);
this.partition.Settings.TestHooks?.CheckpointInjector?.CompactionComplete(this.partition.ErrorHandler);
}
this.Notify();
@ -503,7 +491,7 @@ namespace DurableTask.Netherite.Faster
stopwatch.Start();
this.effectTracker.IsReplaying = true;
await logWorker.ReplayCommitLog(startPosition, this).ConfigureAwait(false);
await logWorker.ReplayCommitLog(startPosition, this);
stopwatch.Stop();
this.effectTracker.IsReplaying = false;
@ -530,7 +518,7 @@ namespace DurableTask.Netherite.Faster
{
foreach (var key in TrackedObjectKey.GetSingletons())
{
var target = (TrackedObject)await this.store.ReadAsync(key, this.effectTracker).ConfigureAwait(false);
var target = (TrackedObject)await this.store.ReadAsync(key, this.effectTracker);
target.OnRecoveryCompleted(this.effectTracker, recoveryCompletedEvent);
}
}
@ -543,7 +531,7 @@ namespace DurableTask.Netherite.Faster
public async ValueTask ReplayUpdate(PartitionUpdateEvent partitionUpdateEvent)
{
await this.effectTracker.ProcessUpdate(partitionUpdateEvent).ConfigureAwait(false);
await this.effectTracker.ProcessUpdate(partitionUpdateEvent);
// update the input queue position if larger
// it can be smaller since the checkpoint can store positions advanced by non-update events
@ -572,7 +560,7 @@ namespace DurableTask.Netherite.Faster
return;
}
await this.effectTracker.ProcessUpdate(partitionUpdateEvent).ConfigureAwait(false);
await this.effectTracker.ProcessUpdate(partitionUpdateEvent);
if (this.replayChecker != null)
{
@ -593,7 +581,7 @@ namespace DurableTask.Netherite.Faster
{
while (true)
{
await Task.Delay(StoreWorker.PokePeriod, this.cancellationToken).ConfigureAwait(false);
await Task.Delay(StoreWorker.PokePeriod, this.cancellationToken);
if (this.cancellationToken.IsCancellationRequested || this.isShuttingDown)
{

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

@ -17,10 +17,12 @@ namespace DurableTask.Netherite.Faster
public abstract Task<(long commitLogPosition, long inputQueuePosition)> RecoverAsync();
public abstract void CompletePending();
public abstract bool CompletePending();
public abstract ValueTask ReadyToCompletePendingAsync();
public abstract void AdjustCacheSize();
public abstract bool TakeFullCheckpoint(long commitLogPosition, long inputQueuePosition, out Guid checkpointGuid);
public abstract Task RemoveObsoleteCheckpoints();
@ -37,6 +39,8 @@ namespace DurableTask.Netherite.Faster
public abstract Task<long> RunCompactionAsync(long target);
public abstract void CheckInvariants();
// perform a query
public abstract Task QueryAsync(PartitionQueryEvent queryEvent, EffectTracker effectTracker);
@ -44,7 +48,7 @@ namespace DurableTask.Netherite.Faster
public abstract Task RunPrefetchSession(IAsyncEnumerable<TrackedObjectKey> keys);
// kick off a read of a tracked object, completing asynchronously if necessary
public abstract void ReadAsync(PartitionReadEvent readEvent, EffectTracker effectTracker);
public abstract void Read(PartitionReadEvent readEvent, EffectTracker effectTracker);
// read a singleton tracked object on the main session and wait for the response (only one of these is executing at a time)
public abstract ValueTask<TrackedObject> ReadAsync(FasterKV.Key key, EffectTracker effectTracker);
@ -60,6 +64,8 @@ namespace DurableTask.Netherite.Faster
public StoreStatistics StoreStats { get; } = new StoreStatistics();
public abstract (double totalSizeMB, int fillPercentage) CacheSizeInfo { get; }
public class StoreStatistics
{
public long Create;

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

@ -185,11 +185,11 @@ namespace DurableTask.Netherite
this.WriteEvent(226, Account, TaskHub, PartitionId, WorkItemType, WorkItemId, InstanceId, Details, ReplacedBy, AppName, ExtensionVersion);
}
[Event(227, Level = EventLevel.Verbose, Version = 2)]
public void InstanceUpdated(string Account, string TaskHub, int PartitionId, string InstanceId, string ExecutionId, string PartitionEventId, string RuntimeStatus, int NewEventCount, int EventCount, string NewEvents, string EventType, int Episode, string AppName, string ExtensionVersion)
[Event(227, Level = EventLevel.Verbose, Version = 3)]
public void InstanceUpdated(string Account, string TaskHub, int PartitionId, string InstanceId, string ExecutionId, string PartitionEventId, string RuntimeStatus, int NewEventCount, int EventCount, long HistorySize, string NewEvents, string EventType, int Episode, string AppName, string ExtensionVersion)
{
SetCurrentThreadActivityId(serviceInstanceId);
this.WriteEvent(227, Account, TaskHub, PartitionId, InstanceId, ExecutionId, PartitionEventId, RuntimeStatus, NewEventCount, EventCount, NewEvents, EventType, Episode, AppName, ExtensionVersion);
this.WriteEvent(227, Account, TaskHub, PartitionId, InstanceId, ExecutionId, PartitionEventId, RuntimeStatus, NewEventCount, EventCount, HistorySize, NewEvents, EventType, Episode, AppName, ExtensionVersion);
}
[Event(228, Level = EventLevel.Verbose, Version = 2)]
@ -250,11 +250,11 @@ namespace DurableTask.Netherite
this.WriteEvent(245, Account, TaskHub, PartitionId, PartitionEventId, ReportedLocalLoad, Pending, Backlog, Remotes, Distribution, AppName, ExtensionVersion);
}
[Event(246, Level = EventLevel.Informational, Version = 2)]
public void PartitionLoadPublished(string Account, string TaskHub, int PartitionId, int WorkItems, int Activities, int Timers, int Requests, int Outbox, long Instances, string NextTimer, string WorkerId, string LatencyTrend, double MissRate, long InputQueuePosition, long CommitLogPosition, string AppName, string ExtensionVersion)
[Event(246, Level = EventLevel.Informational, Version = 3)]
public void PartitionLoadPublished(string Account, string TaskHub, int PartitionId, int WorkItems, int Activities, int Timers, int Requests, int Outbox, long Instances, string NextTimer, string WorkerId, string LatencyTrend, int CachePct, double CacheMB, double MissRate, long InputQueuePosition, long CommitLogPosition, string AppName, string ExtensionVersion)
{
SetCurrentThreadActivityId(serviceInstanceId);
this.WriteEvent(246, Account, TaskHub, PartitionId, WorkItems, Activities, Timers, Requests, Outbox, Instances, NextTimer, WorkerId, LatencyTrend, MissRate, InputQueuePosition, CommitLogPosition, AppName, ExtensionVersion);
this.WriteEvent(246, Account, TaskHub, PartitionId, WorkItems, Activities, Timers, Requests, Outbox, Instances, NextTimer, WorkerId, LatencyTrend, CachePct, CacheMB, MissRate, InputQueuePosition, CommitLogPosition, AppName, ExtensionVersion);
}
[Event(247, Level = EventLevel.Verbose, Version = 2)]
@ -378,6 +378,13 @@ namespace DurableTask.Netherite
this.WriteEvent(267, Account, TaskHub, PartitionId, Details, AppName, ExtensionVersion);
}
[Event(268, Level = EventLevel.Informational, Version = 1)]
public void FasterCompactionProgress(string Account, string TaskHub, int PartitionId, string Details, string Operation, long Begin, long SafeReadOnly, long Tail, long MinimalSize, long CompactionAreaSize, long ElapsedMs, string AppName, string ExtensionVersion)
{
SetCurrentThreadActivityId(serviceInstanceId);
this.WriteEvent(266, Account, TaskHub, PartitionId, Details, Operation, Begin, SafeReadOnly, Tail, MinimalSize, CompactionAreaSize, ElapsedMs, AppName, ExtensionVersion);
}
// ----- EventHubs Transport
[Event(270, Level = EventLevel.Informational, Version = 1)]

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

@ -69,6 +69,7 @@ namespace DurableTask.Netherite
OrchestrationStatus runtimeStatus,
int totalEventCount,
List<HistoryEvent> newEvents,
long historySize,
int episode)
{
if (this.logLevelLimit <= LogLevel.Debug)
@ -120,11 +121,11 @@ namespace DurableTask.Netherite
(long commitLogPosition, string eventId) = EventTraceContext.Current;
string prefix = commitLogPosition > 0 ? $".{commitLogPosition:D10} " : "";
this.logger.LogDebug("Part{partition:D2}{prefix} Updated instance instanceId={instanceId} executionId={executionId} partitionEventId={partitionEventId} runtimeStatus={runtimeStatus} numNewEvents={numNewEvents} totalEventCount={totalEventCount} eventNames={eventNames} eventType={eventType} episode={episode}",
this.partitionId, prefix, instanceId, executionId, partitionEventId, runtimeStatus, numNewEvents, totalEventCount, eventNames, eventType, episode);
this.logger.LogDebug("Part{partition:D2}{prefix} Updated instance instanceId={instanceId} executionId={executionId} partitionEventId={partitionEventId} runtimeStatus={runtimeStatus} numNewEvents={numNewEvents} totalEventCount={totalEventCount} historySize={historySize} eventNames={eventNames} eventType={eventType} episode={episode}",
this.partitionId, prefix, instanceId, executionId, partitionEventId, runtimeStatus, numNewEvents, totalEventCount, historySize, eventNames, eventType, episode);
}
this.etw?.InstanceUpdated(this.account, this.taskHub, this.partitionId, instanceId, executionId, partitionEventId, runtimeStatus.ToString(), numNewEvents, totalEventCount, eventNames, eventType, episode, TraceUtils.AppName, TraceUtils.ExtensionVersion);
this.etw?.InstanceUpdated(this.account, this.taskHub, this.partitionId, instanceId, executionId, partitionEventId, runtimeStatus.ToString(), numNewEvents, totalEventCount, historySize, eventNames, eventType, episode, TraceUtils.AppName, TraceUtils.ExtensionVersion);
}
}

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

@ -90,16 +90,21 @@ namespace DurableTask.Netherite
}
public void TraceError(string message, Exception exception)
{
this.TraceError(message, exception.ToString());
}
public void TraceError(string message, string details)
{
if (this.logLevelLimit <= LogLevel.Error)
{
if (this.Logger.IsEnabled(LogLevel.Error))
{
this.Logger.LogError("NetheriteOrchestrationService !!! {message}: {exception}", message, exception);
this.Logger.LogError("NetheriteOrchestrationService !!! {message}: {details}", message, details);
}
if (EtwSource.Log.IsEnabled())
{
EtwSource.Log.OrchestrationServiceError(this.StorageAccountName, message, exception.ToString(), this.taskHub, this.workerName, TraceUtils.AppName, TraceUtils.ExtensionVersion);
EtwSource.Log.OrchestrationServiceError(this.StorageAccountName, message, details, this.taskHub, this.workerName, TraceUtils.AppName, TraceUtils.ExtensionVersion);
}
}
}

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

@ -44,10 +44,10 @@ namespace DurableTask.Netherite
{
if (this.logLevelLimit <= LogLevel.Information)
{
this.logger.LogInformation("Part{partition:D2} Publishing LoadInfo WorkItems={workItems} Activities={activities} Timers={timers} Requests={requests} Outbox={outbox} Instances={instances} Wakeup={wakeup} WorkerId={workerId} LatencyTrend={latencyTrend} MissRate={missRate} InputQueuePosition={inputQueuePosition} CommitLogPosition={commitLogPosition}",
this.partitionId, info.WorkItems, info.Activities, info.Timers, info.Requests, info.Outbox, info.Instances, info.Wakeup, info.WorkerId, info.LatencyTrend, info.MissRate, info.InputQueuePosition, info.CommitLogPosition);
this.logger.LogInformation("Part{partition:D2} Publishing LoadInfo WorkItems={workItems} Activities={activities} Timers={timers} Requests={requests} Outbox={outbox} Instances={instances} Wakeup={wakeup} WorkerId={workerId} LatencyTrend={latencyTrend} CachePct={cachePct} CacheMB={cacheMB} MissRate={missRate} InputQueuePosition={inputQueuePosition} CommitLogPosition={commitLogPosition}",
this.partitionId, info.WorkItems, info.Activities, info.Timers, info.Requests, info.Outbox, info.Instances, info.Wakeup, info.WorkerId, info.LatencyTrend, info.CachePct, info.CacheMB, info.MissRate, info.InputQueuePosition, info.CommitLogPosition);
EtwSource.Log.PartitionLoadPublished(this.account, this.taskHub, this.partitionId, info.WorkItems, info.Activities, info.Timers, info.Requests, info.Outbox, info.Instances, info.Wakeup?.ToString("o") ?? "", info.WorkerId, info.LatencyTrend, info.MissRate, info.InputQueuePosition, info.CommitLogPosition, TraceUtils.AppName, TraceUtils.ExtensionVersion);
EtwSource.Log.PartitionLoadPublished(this.account, this.taskHub, this.partitionId, info.WorkItems, info.Activities, info.Timers, info.Requests, info.Outbox, info.Instances, info.Wakeup?.ToString("o") ?? "", info.WorkerId, info.LatencyTrend, info.CachePct, info.CacheMB, info.MissRate, info.InputQueuePosition, info.CommitLogPosition, TraceUtils.AppName, TraceUtils.ExtensionVersion);
}
}

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

@ -142,6 +142,8 @@ namespace DurableTask.Netherite.EventHubs
{
try
{
this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} is readying next startup (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, c.Incarnation);
await Task.Delay(-1, prior.ErrorHandler.Token);
}
catch (OperationCanceledException)
@ -153,7 +155,20 @@ namespace DurableTask.Netherite.EventHubs
// we are now becoming the current incarnation
this.currentIncarnation = prior.Next;
this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} is restarting partition (incarnation {incarnation}) soon", this.eventHubName, this.eventHubPartition, c.Incarnation);
await Task.Delay(TimeSpan.FromSeconds(12), this.eventProcessorShutdown.Token);
// we wait at most 20 seconds for the previous partition to terminate cleanly
int tries = 4;
var timeout = TimeSpan.FromSeconds(5);
while (!await prior.ErrorHandler.WaitForTermination(timeout))
{
this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} partition (incarnation {incarnation}) is still waiting for PartitionShutdown of previous incarnation", this.eventHubName, this.eventHubPartition, c.Incarnation);
if (--tries == 0)
{
break;
}
}
}
}
@ -164,8 +179,8 @@ namespace DurableTask.Netherite.EventHubs
return null;
}
// start the next incarnation, will be on standby until after the current one is terminated
c.Next = Task.Run(() => this.StartPartitionAsync(c));
// start the next incarnation task, will be on standby until after the current one is terminated
c.Next = this.StartPartitionAsync(c);
try
{

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

@ -300,8 +300,9 @@ namespace DurableTask.Netherite.EventHubs
var pos = this.parameters.StartPositions[int.Parse(s)];
if (pos > 0)
{
this.traceHelper.LogDebug("InitialOffsetProvider: partitions/{partitionId} EventPosition.FromSequenceNumber({offset}, inclusive:false)", s, pos - 1);
return EventPosition.FromSequenceNumber(pos - 1, inclusive: false);
// to work around bug in EH, ask for the most recently received packet to be delivered again
this.traceHelper.LogDebug("InitialOffsetProvider: partitions/{partitionId} EventPosition.FromSequenceNumber({offset}, inclusive:true)", s, pos - 1);
return EventPosition.FromSequenceNumber(pos - 1, inclusive: true);
}
else
{
@ -336,8 +337,7 @@ namespace DurableTask.Netherite.EventHubs
this.traceHelper,
this.settings.WorkerId);
var thread = new Thread(() => this.scriptedEventProcessorHost.StartEventProcessing(this.settings, this.partitionScript));
thread.Name = "ScriptedEventProcessorHost";
var thread = TrackedThreads.MakeTrackedThread(() => this.scriptedEventProcessorHost.StartEventProcessing(this.settings, this.partitionScript), "ScriptedEventProcessorHost");
thread.Start();
}
}
@ -537,7 +537,7 @@ namespace DurableTask.Netherite.EventHubs
if (clientEvent != null && clientEvent.ClientId == this.ClientId)
{
this.traceHelper.LogTrace("Client{clientId}.ch{index} receiving event {evt} id={eventId}]", Client.GetShortId(this.ClientId), index, clientEvent, clientEvent.EventIdString);
await channelWriter.WriteAsync(clientEvent);
await channelWriter.WriteAsync(clientEvent, this.shutdownSource.Token);
}
}
catch (Exception)

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

@ -267,9 +267,9 @@ namespace DurableTask.Netherite.EventHubs
{
var tcs = new TaskCompletionSource<object>(TaskCreationOptions.RunContinuationsAsynchronously);
var registration = this.shutdownSource.Token.Register(() =>
{
{
tcs.TrySetResult(true);
});
});
await tcs.Task;
registration.Dispose();
}
@ -358,7 +358,7 @@ namespace DurableTask.Netherite.EventHubs
if (partitionEvent != null)
{
this.host.logger.LogTrace("PartitionInstance {eventHubName}/{eventHubPartition}({incarnation}) received packet #{seqno} ({size} bytes) {event}", this.host.eventHubPath, this.partitionId, this.Incarnation, seqno, eventDatum.Body.Count, partitionEvent);
this.host.logger.LogTrace("PartitionInstance {eventHubName}/{eventHubPartition}({incarnation}) received packet #{seqno} ({size} bytes) {event} id={eventId}", this.host.eventHubPath, this.partitionId, this.Incarnation, seqno, eventDatum.Body.Count, partitionEvent, partitionEvent.EventIdString);
}
else
{

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

@ -132,7 +132,7 @@ namespace DurableTask.Netherite.Emulated
if (epoch > 0)
{
if (this.settings.TestHooks != null && this.settings.TestHooks.FaultInjector == null)
if (this.settings.TestHooks?.FaultInjectionActive != true)
{
this.settings.TestHooks.Error("MemoryTransport", "Unexpected partition termination");
}
@ -191,7 +191,7 @@ namespace DurableTask.Netherite.Emulated
var errorHandler = this.host.CreateErrorHandler((uint)i);
if (this.faultInjector != null)
{
errorHandler.Token.Register(() => this.RecoveryHandler(epoch));
errorHandler.OnShutdown += () => this.RecoveryHandler(epoch);
}
var nextInputQueuePosition = await partitions[i].CreateOrRestoreAsync(errorHandler, 0);
@ -290,6 +290,11 @@ namespace DurableTask.Netherite.Emulated
{
queue.Send(clientEvent);
}
else
{
// client does not exist, can happen after recovery
DurabilityListeners.ConfirmDurable(clientEvent);
}
}
else if (evt is PartitionEvent partitionEvent)
{

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

@ -34,8 +34,7 @@ namespace DurableTask.Netherite
public void Start(string name)
{
var thread = new Thread(this.ExpirationCheckLoop);
thread.Name = name;
var thread = TrackedThreads.MakeTrackedThread(this.ExpirationCheckLoop, name);
this.name = name;
thread.Start();
}
@ -124,9 +123,16 @@ namespace DurableTask.Netherite
{
this.tracer?.Invoke($"starting {this.name} batch size={batch.Count} first=({firstInBatch.due:o},{firstInBatch.id}) next=({nextAfterBatch.due:o},{nextAfterBatch.id})");
// it is expected that the handler catches
// all exceptions, since it has more meaningful ways to report errors
this.handler(batch);
try
{
this.handler(batch);
}
catch
{
// it is expected that the handler catches
// all exceptions, since it has more meaningful ways to report errors
}
batch.Clear();
this.tracer?.Invoke($"completed {this.name} batch size={batch.Count} first=({firstInBatch.due:o},{firstInBatch.id}) next=({nextAfterBatch.due:o},{nextAfterBatch.id})");

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

@ -6,6 +6,7 @@ namespace DurableTask.Netherite
using System;
using System.Threading;
using System.Threading.Tasks;
using DurableTask.Core.Common;
using Microsoft.Azure.Storage;
using Microsoft.Azure.Storage.Blob;
using Microsoft.Azure.Storage.Blob.Protocol;
@ -59,61 +60,76 @@ namespace DurableTask.Netherite
/// <param name="e">The storage exception.</param>
/// <param name="token">The cancellation token that was passed to the storage request.</param>
/// <returns>Whether this is a transient storage exception.</returns>
public static bool IsTransientStorageError(StorageException e, CancellationToken token)
public static bool IsTransientStorageError(Exception exception, CancellationToken token)
{
// Transient error codes as documented at https://docs.microsoft.com/en-us/azure/architecture/best-practices/retry-service-specific#azure-storage
if ( (e.RequestInformation.HttpStatusCode == 408) //408 Request Timeout
|| (e.RequestInformation.HttpStatusCode == 429) //429 Too Many Requests
|| (e.RequestInformation.HttpStatusCode == 500) //500 Internal Server Error
|| (e.RequestInformation.HttpStatusCode == 502) //502 Bad Gateway
|| (e.RequestInformation.HttpStatusCode == 503) //503 Service Unavailable
|| (e.RequestInformation.HttpStatusCode == 504)) //504 Gateway Timeout
if (exception is StorageException e)
{
return true;
// Transient error codes as documented at https://docs.microsoft.com/en-us/azure/architecture/best-practices/retry-service-specific#azure-storage
if ((e.RequestInformation.HttpStatusCode == 408) //408 Request Timeout
|| (e.RequestInformation.HttpStatusCode == 429) //429 Too Many Requests
|| (e.RequestInformation.HttpStatusCode == 500) //500 Internal Server Error
|| (e.RequestInformation.HttpStatusCode == 502) //502 Bad Gateway
|| (e.RequestInformation.HttpStatusCode == 503) //503 Service Unavailable
|| (e.RequestInformation.HttpStatusCode == 504)) //504 Gateway Timeout
{
return true;
}
// Empirically observed transient cancellation exceptions that are not application initiated
if (e.InnerException is OperationCanceledException && !token.IsCancellationRequested)
{
return true;
}
// Empirically observed timeouts on synchronous calls
if (e.InnerException is TimeoutException)
{
return true;
}
// Empirically observed transient exception
// ('An existing connection was forcibly closed by the remote host')
if (e.InnerException is System.Net.Http.HttpRequestException
&& e.InnerException?.InnerException is System.IO.IOException)
{
return true;
}
}
// Empirically observed transient cancellation exceptions that are not application initiated
if (e.InnerException is OperationCanceledException && !token.IsCancellationRequested)
if (exception is System.IO.IOException && exception.InnerException is System.Net.Sockets.SocketException)
{
return true;
}
// Empirically observed transient exception
// ('An existing connection was forcibly closed by the remote host')
if (e.InnerException is System.Net.Http.HttpRequestException
&& e.InnerException?.InnerException is System.IO.IOException)
{
return true;
return true; // empirically observed
}
return false;
}
/// <summary>
/// Checks whether the given storage exception is a timeout exception.
/// Checks whether the given exception is a timeout exception.
/// </summary>
/// <param name="e">The storage exception.</param>
/// <param name="e">The exception.</param>
/// <returns>Whether this is a timeout storage exception.</returns>
public static bool IsTimeout(StorageException e)
public static bool IsTimeout(Exception exception)
{
return (e.RequestInformation.HttpStatusCode == 408); //408 Request Timeout
return exception is System.TimeoutException
|| (exception is StorageException e && e.RequestInformation?.HttpStatusCode == 408); //408 Request Timeout
}
// Lease error codes are documented at https://docs.microsoft.com/en-us/rest/api/storageservices/lease-blob
public static bool LeaseConflictOrExpired(StorageException e)
{
return (e.RequestInformation.HttpStatusCode == 409) || (e.RequestInformation.HttpStatusCode == 412);
return (e.RequestInformation?.HttpStatusCode == 409) || (e.RequestInformation?.HttpStatusCode == 412);
}
public static bool LeaseConflict(StorageException e)
{
return (e.RequestInformation.HttpStatusCode == 409);
return (e.RequestInformation?.HttpStatusCode == 409);
}
public static bool LeaseExpired(StorageException e)
{
return (e.RequestInformation.HttpStatusCode == 412);
return (e.RequestInformation?.HttpStatusCode == 412);
}
public static bool CannotDeleteBlobWithLease(StorageException e)
@ -123,13 +139,13 @@ namespace DurableTask.Netherite
public static bool BlobDoesNotExist(StorageException e)
{
var information = e.RequestInformation.ExtendedErrorInformation;
return (e.RequestInformation.HttpStatusCode == 404) && (information.ErrorCode.Equals(BlobErrorCodeStrings.BlobNotFound));
var information = e.RequestInformation?.ExtendedErrorInformation;
return (e.RequestInformation?.HttpStatusCode == 404) && (information.ErrorCode.Equals(BlobErrorCodeStrings.BlobNotFound));
}
public static bool BlobAlreadyExists(StorageException e)
{
return (e.RequestInformation.HttpStatusCode == 409);
return (e.RequestInformation?.HttpStatusCode == 409);
}
}
}

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

@ -0,0 +1,125 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
namespace DurableTask.Netherite
{
using System;
using System.Collections.Generic;
using System.Diagnostics;
using System.Linq;
using DurableTask.Core;
using DurableTask.Core.History;
/// <summary>
/// Functionality for estimating memory size.
/// </summary>
class SizeUtils
{
public static long GetEstimatedSize(OrchestrationInstance instance)
{
return instance == null ? 0 : 32 + 2 * (instance.InstanceId?.Length ?? 0 + instance.ExecutionId?.Length ?? 0);
}
public static long GetEstimatedSize(ParentInstance p)
{
return p == null ? 0 : 36 + 2 * (p.Name?.Length ?? 0 + p.Version?.Length ?? 0) + GetEstimatedSize(p.OrchestrationInstance);
}
public static long GetEstimatedSize(OrchestrationState state)
{
//DateTime CompletedTime;
long sum = 0;
if (state != null)
{
sum += 120;
sum += 2 * ((state.Status?.Length ?? 0) + (state.Output?.Length ?? 0) + (state.Name?.Length ?? 0) + (state.Input?.Length ?? 0) + (state.Version?.Length ?? 0));
sum += GetEstimatedSize(state.OrchestrationInstance) + GetEstimatedSize(state.ParentInstance);
}
return sum;
}
public static long GetEstimatedSize(HistoryEvent historyEvent)
{
long estimate = 5*8; // estimated size of base class
void AddString(string s) => estimate += 8 + ((s == null) ? 0 : 2 * s.Length);
switch (historyEvent)
{
case ContinueAsNewEvent continueAsNewEvent:
AddString(continueAsNewEvent.Result);
break;
case EventRaisedEvent eventRaisedEvent:
AddString(eventRaisedEvent.Input);
AddString(eventRaisedEvent.Name);
break;
case EventSentEvent eventSentEvent:
AddString(eventSentEvent.InstanceId);
AddString(eventSentEvent.Name);
AddString(eventSentEvent.Input);
break;
case ExecutionCompletedEvent executionCompletedEvent:
AddString(executionCompletedEvent.Result);
estimate += 8;
break;
case ExecutionStartedEvent executionStartedEvent:
AddString(executionStartedEvent.Input);
AddString(executionStartedEvent.Name);
AddString(executionStartedEvent.OrchestrationInstance.InstanceId);
AddString(executionStartedEvent.OrchestrationInstance.ExecutionId);
estimate += 8 + (executionStartedEvent.Tags == null ? 0
: executionStartedEvent.Tags.Select(kvp => 20 + 2 * (kvp.Key.Length + kvp.Value.Length)).Sum());
AddString(executionStartedEvent.Version);
AddString(executionStartedEvent.ParentInstance?.OrchestrationInstance.InstanceId);
AddString(executionStartedEvent.ParentInstance?.OrchestrationInstance.ExecutionId);
estimate += 8;
break;
case ExecutionTerminatedEvent executionTerminatedEvent:
AddString(executionTerminatedEvent.Input);
break;
case GenericEvent genericEvent:
AddString(genericEvent.Data);
break;
case OrchestratorCompletedEvent:
break;
case OrchestratorStartedEvent:
break;
case SubOrchestrationInstanceCompletedEvent subOrchestrationInstanceCompletedEvent:
estimate += 8;
AddString(subOrchestrationInstanceCompletedEvent.Result);
break;
case SubOrchestrationInstanceCreatedEvent subOrchestrationInstanceCreatedEvent:
AddString(subOrchestrationInstanceCreatedEvent.Input);
AddString(subOrchestrationInstanceCreatedEvent.Input);
AddString(subOrchestrationInstanceCreatedEvent.Name);
AddString(subOrchestrationInstanceCreatedEvent.Version);
break;
case SubOrchestrationInstanceFailedEvent subOrchestrationInstanceFailedEvent:
estimate += 8;
AddString(subOrchestrationInstanceFailedEvent.Reason);
AddString(subOrchestrationInstanceFailedEvent.Details);
break;
case TaskCompletedEvent taskCompletedEvent:
estimate += 8;
AddString(taskCompletedEvent.Result);
break;
case TaskFailedEvent taskFailedEvent:
estimate += 8;
AddString(taskFailedEvent.Reason);
AddString(taskFailedEvent.Details);
break;
case TaskScheduledEvent taskScheduledEvent:
AddString(taskScheduledEvent.Input);
AddString(taskScheduledEvent.Name);
AddString(taskScheduledEvent.Version);
break;
case TimerCreatedEvent timerCreatedEvent:
estimate += 8;
break;
case TimerFiredEvent timerFiredEvent:
estimate += 16;
break;
}
return estimate;
}
}
}

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

@ -0,0 +1,51 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
namespace DurableTask.Netherite
{
using System;
using System.Collections.Concurrent;
using System.Collections.Generic;
using System.Linq;
using System.Text.RegularExpressions;
using System.Threading;
/// <summary>
/// Functionality for estimating memory size.
/// </summary>
class TrackedThreads
{
readonly static ConcurrentDictionary<int, Thread> threads = new ConcurrentDictionary<int, Thread>();
public static Thread MakeTrackedThread(Action action, string name)
{
Thread thread = null;
thread = new Thread(ThreadStart) { Name = name };
void ThreadStart()
{
threads.TryAdd(thread.ManagedThreadId, thread);
try
{
action();
}
finally
{
threads.TryRemove(thread.ManagedThreadId, out _);
}
}
return thread;
}
public static int NumberThreads => threads.Count;
public static string GetThreadNames()
{
return string.Join(",", threads
.Values
.GroupBy((thread) => thread.Name)
.OrderByDescending(group => group.Count())
.Select(group => $"{group.Key}(x{group.Count()})"));
}
}
}

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

@ -0,0 +1,56 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
namespace DurableTask.Netherite.Tests
{
using System;
using System.Collections.Generic;
using System.Linq;
using System.Threading;
using System.Threading.Tasks;
static class Common
{
public static async Task<List<S>> ParallelForEachAsync<T,S>(this IEnumerable<T> items, int maxConcurrency, bool useThreadpool, Func<T, Task<S>> action)
{
List<Task<S>> tasks;
if (items is ICollection<T> itemCollection)
{
tasks = new List<Task<S>>(itemCollection.Count);
}
else
{
tasks = new List<Task<S>>();
}
using var semaphore = new SemaphoreSlim(maxConcurrency);
foreach (T item in items)
{
tasks.Add(InvokeThrottledAction(item, action, semaphore, useThreadpool));
}
await Task.WhenAll(tasks);
return tasks.Select(t => t.Result).ToList();
}
static async Task<S> InvokeThrottledAction<T,S>(T item, Func<T, Task<S>> action, SemaphoreSlim semaphore, bool useThreadPool)
{
await semaphore.WaitAsync();
try
{
if (useThreadPool)
{
return await Task.Run(() => action(item));
}
else
{
return await action(item);
}
}
finally
{
semaphore.Release();
}
}
}
}

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

@ -21,7 +21,7 @@ namespace DurableTask.Netherite.Tests
using Xunit.Abstractions;
[Collection("NetheriteTests")]
[Trait("AnyTransport", "true")]
[Trait("AnyTransport", "false")]
public partial class ConcurrentTests : IDisposable
{
ITestOutputHelper outputHelper;
@ -31,22 +31,38 @@ namespace DurableTask.Netherite.Tests
{
this.outputHelper = outputHelper;
this.settings = TestConstants.GetNetheriteOrchestrationServiceSettings();
string timestamp = DateTime.UtcNow.ToString("yyyyMMdd-HHmmss-fffffff");
this.settings.HubName = $"ConcurrentTests-{Guid.NewGuid().ToString("n")}";
}
public void Dispose()
{
if (this.settings.TestHooks.CacheDebugger != null)
{
this.outputHelper.WriteLine("CACHEDEBUGGER DUMP: --------------------------------------------------------------------------------------------------------------");
foreach (var line in this.settings.TestHooks.CacheDebugger.Dump())
{
this.outputHelper.WriteLine(line);
}
}
this.outputHelper = null;
}
async Task WaitForCompletion(List<(string, Task)> tests)
async Task WaitForCompletion(List<(string, Task)> tests, TimeSpan timeout)
{
var alldone = Task.WhenAll(tests.Select(x => x.Item2));
Stopwatch stopwatch = new Stopwatch();
stopwatch.Start();
TimeSpan timeout = TimeSpan.FromMinutes(5);
string errorInTestHooks = null;
while (!alldone.IsCompleted)
this.settings.TestHooks.OnError += (string message) =>
{
this.outputHelper?.WriteLine(message);
errorInTestHooks ??= message;
};
while (!alldone.IsCompleted && errorInTestHooks == null)
{
string incomplete = string.Join(", ", tests.Where(x => !x.Item2.IsCompleted).Select(x => x.Item1));
Trace.WriteLine($"TestProgress: Waiting for {incomplete}");
@ -61,63 +77,118 @@ namespace DurableTask.Netherite.Tests
await Task.WhenAny(alldone, checkAgain);
}
Assert.Null(errorInTestHooks);
await Task.WhenAll(alldone); // to propagate exceptions
}
[Fact]
public async Task EachScenarioOnce()
[Theory]
[InlineData(false)]
[InlineData(true)]
public async Task EachScenarioOnce(bool restrictMemory)
{
using var _ = TestOrchestrationClient.WithExtraTime(TimeSpan.FromMinutes(1));
using var fixture = await SingleHostFixture.StartNew(this.settings, false, TimeSpan.FromMinutes(3), (msg) => this.outputHelper?.WriteLine(msg));
using var _ = TestOrchestrationClient.WithExtraTime(TimeSpan.FromMinutes(restrictMemory ? 10 : 5));
using var fixture = await SingleHostFixture.StartNew(this.settings, useCacheDebugger: true, useReplayChecker: true, restrictMemory ? (int?) 0 : null, TimeSpan.FromMinutes(5), (msg) => this.outputHelper?.WriteLine(msg));
var scenarios = new ScenarioTests(fixture, this.outputHelper);
var tests = scenarios.StartAllScenarios().ToList();
await this.WaitForCompletion(tests);
var tests = scenarios.StartAllScenarios(includeTimers: !restrictMemory, includeLarge: true).ToList();
await this.WaitForCompletion(tests, TimeSpan.FromMinutes(restrictMemory ? 10 : 5));
}
[Fact]
public async Task SmallOnesWithReplayCheck()
[Theory]
[InlineData(false, false, 4)]
[InlineData(false, true, 4)]
[InlineData(true, false, 4)]
[InlineData(true, true, 4)]
[InlineData(false, false, 20)]
[InlineData(false, true, 20)]
[InlineData(true, false, 20)]
public async Task ScaleSmallScenarios(bool useReplayChecker, bool restrictMemory, int multiplicity)
{
using var _ = TestOrchestrationClient.WithExtraTime(TimeSpan.FromMinutes(1));
using var fixture = await SingleHostFixture.StartNew(this.settings, true, TimeSpan.FromMinutes(3), (msg) => this.outputHelper?.WriteLine(msg));
var scenarios = new ScenarioTests(fixture, this.outputHelper);
var orchestrationTimeout = TimeSpan.FromMinutes((restrictMemory ? 10 : 5) + multiplicity * (restrictMemory ? 0.5 : 0.1));
var startupTimeout = TimeSpan.FromMinutes(TransportConnectionString.IsEmulatorSpecification(this.settings.ResolvedTransportConnectionString) ? 1 : 3.5);
var testTimeout = orchestrationTimeout + TimeSpan.FromMinutes(multiplicity * 0.2);
var shutDownTimeout = TimeSpan.FromMinutes(TransportConnectionString.IsEmulatorSpecification(this.settings.ResolvedTransportConnectionString) ? 0.1 : 3);
var totalTimeout = startupTimeout + testTimeout + shutDownTimeout;
var tests = scenarios.StartAllScenarios(false, false).ToList();
using var _ = TestOrchestrationClient.WithExtraTime(orchestrationTimeout);
await this.WaitForCompletion(tests);
async Task RunAsync()
{
Trace.WriteLine($"TestProgress: Started RunAsync");
using (var fixture = await SingleHostFixture.StartNew(
this.settings,
true,
useReplayChecker,
restrictMemory ? (int?)0 : null,
startupTimeout,
(msg) => this.outputHelper?.WriteLine(msg)))
{
var scenarios = new ScenarioTests(fixture, this.outputHelper);
var tests = new List<(string, Task)>();
for (int i = 0; i < multiplicity; i++)
{
foreach((string name, Task task) in scenarios.StartAllScenarios(false, false))
{
Trace.WriteLine($"TestProgress: Adding {name}");
tests.Add((name,task));
}
}
await this.WaitForCompletion(tests, testTimeout);
Trace.WriteLine($"TestProgress: Shutting Down");
}
Trace.WriteLine($"TestProgress: Completed RunAsync");
}
var task = Task.Run(RunAsync);
var timeoutTask = Task.Delay(totalTimeout);
await Task.WhenAny(task, timeoutTask);
Assert.True(task.IsCompleted);
await task;
}
[Fact]
public async Task SmallOnesTimesTwenty()
[Theory]
[InlineData(1)]
[InlineData(2)]
[InlineData(3)]
[InlineData(4)]
[InlineData(5)]
[InlineData(6)]
[InlineData(7)]
[InlineData(8)]
public async Task ReproHangingReads(int sequenceNumber)
{
using var _ = TestOrchestrationClient.WithExtraTime(TimeSpan.FromMinutes(1));
using var fixture = await SingleHostFixture.StartNew(this.settings, false, TimeSpan.FromMinutes(3), (msg) => this.outputHelper?.WriteLine(msg));
// running a single test is usually not enough to repro, so we run the same test multiple times
this.outputHelper.WriteLine($"starting test {sequenceNumber}");
// disable checkpoints since they are not needed to trigger the bug
this.settings.MaxNumberBytesBetweenCheckpoints = 1024L * 1024 * 1024 * 1024;
this.settings.MaxNumberEventsBetweenCheckpoints = 10000000000L;
this.settings.IdleCheckpointFrequencyMs = (long)TimeSpan.FromDays(1).TotalMilliseconds;
this.settings.PartitionCount = 4;
using var _ = TestOrchestrationClient.WithExtraTime(TimeSpan.FromMinutes(3));
using var fixture = await SingleHostFixture.StartNew(this.settings, true, false, 0, TimeSpan.FromMinutes(5), (msg) => this.outputHelper?.WriteLine(msg));
this.settings.TestHooks.CacheDebugger.EnableSizeChecking = false;
var scenarios = new ScenarioTests(fixture, this.outputHelper);
var tests = new List<(string, Task)>();
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
tests.AddRange(scenarios.StartAllScenarios(false, false));
await this.WaitForCompletion(tests);
for (int i = 0; i < 20; i++)
{
tests.AddRange(scenarios.StartAllScenarios(false, false));
}
await this.WaitForCompletion(tests, TimeSpan.FromMinutes(10));
}
}
}

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

@ -13,6 +13,7 @@ namespace DurableTask.Netherite.Tests
using System.Threading.Tasks;
using DurableTask.Core;
using DurableTask.Core.History;
using DurableTask.Netherite.Faster;
using Microsoft.Extensions.Logging;
using Newtonsoft.Json;
using Xunit;
@ -26,7 +27,14 @@ namespace DurableTask.Netherite.Tests
readonly ILoggerFactory loggerFactory;
readonly XunitLoggerProvider provider;
readonly Action<string> output;
readonly NetheriteOrchestrationServiceSettings settings;
readonly CancellationTokenSource cts;
readonly CacheDebugger cacheDebugger;
ITestOutputHelper outputHelper;
string errorInTestHooks;
const int extraLogEntrySize = 96; // since v2 Faster puts extra stuff in the log. Empirically determined.
public FasterPartitionTests(ITestOutputHelper outputHelper)
{
@ -39,6 +47,18 @@ namespace DurableTask.Netherite.Tests
this.traceListener = new SingleHostFixture.TestTraceListener();
Trace.Listeners.Add(this.traceListener);
this.traceListener.Output = this.output;
this.settings = TestConstants.GetNetheriteOrchestrationServiceSettings();
string timestamp = DateTime.UtcNow.ToString("yyyyMMdd-HHmmss-fffffff");
this.settings.HubName = $"FasterPartitionTest-{timestamp}";
this.settings.ResolvedTransportConnectionString = "MemoryF";
this.cts = new CancellationTokenSource();
this.cacheDebugger = this.settings.TestHooks.CacheDebugger = new Faster.CacheDebugger(this.settings.TestHooks);
this.settings.TestHooks.OnError += (message) =>
{
this.output($"TESTHOOKS: {message}");
this.errorInTestHooks = this.errorInTestHooks ?? message;
this.cts.Cancel();
};
}
public void Dispose()
@ -47,96 +67,103 @@ namespace DurableTask.Netherite.Tests
Trace.Listeners.Remove(this.traceListener);
}
enum CheckpointFrequency
{
None,
Default,
Frequent,
Crazy
}
void SetCheckpointFrequency(CheckpointFrequency frequency)
{
switch (frequency)
{
case CheckpointFrequency.None:
this.settings.MaxNumberBytesBetweenCheckpoints = 1024L * 1024 * 1024 * 1024;
this.settings.MaxNumberEventsBetweenCheckpoints = 10000000000L;
this.settings.IdleCheckpointFrequencyMs = (long)TimeSpan.FromDays(1).TotalMilliseconds;
return;
case CheckpointFrequency.Frequent:
this.settings.MaxNumberEventsBetweenCheckpoints = 100;
return;
case CheckpointFrequency.Crazy:
this.settings.MaxNumberEventsBetweenCheckpoints = 1;
return;
default:
return;
}
}
async Task<(IOrchestrationService orchestrationService, TaskHubClient client)> StartService(bool recover, Type orchestrationType, Type activityType = null)
{
var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceClient = (IOrchestrationServiceClient)service;
await orchestrationService.CreateAsync();
await orchestrationService.StartAsync();
Assert.Equal(this.settings.PartitionCount, (int)service.NumberPartitions);
var worker = new TaskHubWorker(service);
var client = new TaskHubClient(service);
worker.AddTaskOrchestrations(orchestrationType);
if (activityType != null)
{
worker.AddTaskActivities(activityType);
}
await worker.StartAsync();
return (service, client);
}
/// <summary>
/// Create a partition and then restore it.
/// </summary>
[Fact]
public async Task CreateThenRestore()
{
var settings = TestConstants.GetNetheriteOrchestrationServiceSettings();
settings.ResolvedTransportConnectionString = "MemoryF";
settings.PartitionCount = 1;
settings.HubName = $"{TestConstants.TaskHubName}-{Guid.NewGuid()}";
this.settings.PartitionCount = 1;
var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloInline);
{
// start the service
var service = new NetheriteOrchestrationService(settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceClient = (IOrchestrationServiceClient)service;
await orchestrationService.CreateAsync();
await orchestrationService.StartAsync();
var host = (TransportAbstraction.IHost)service;
Assert.Equal(1u, service.NumberPartitions);
var worker = new TaskHubWorker(service);
var client = new TaskHubClient(service);
worker.AddTaskOrchestrations(orchestrationType);
await worker.StartAsync();
var (service, client) = await this.StartService(recover: false, orchestrationType);
// do orchestration
var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "0", "0");
await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(20));
// stop the service
await orchestrationService.StopAsync();
await service.StopAsync();
}
{
// start the service
var service = new NetheriteOrchestrationService(settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceClient = (IOrchestrationServiceClient)service;
await orchestrationService.CreateAsync();
await orchestrationService.StartAsync();
var host = (TransportAbstraction.IHost)service;
Assert.Equal(1u, service.NumberPartitions);
var client = new TaskHubClient(orchestrationServiceClient);
var (service, client) = await this.StartService(recover: true, orchestrationType);
var orchestrationState = await client.GetOrchestrationStateAsync("0");
Assert.Equal(OrchestrationStatus.Completed, orchestrationState.OrchestrationStatus);
Assert.Equal(OrchestrationStatus.Completed, orchestrationState?.OrchestrationStatus);
// stop the service
await orchestrationService.StopAsync();
await service.StopAsync();
}
}
/// <summary>
/// Run a number of orchestrations that requires more memory than available for FASTER
/// </summary>
[Fact(Skip ="CachedDebugger will only work for Faster v2")]
[Fact()]
public async Task LimitedMemory()
{
var settings = TestConstants.GetNetheriteOrchestrationServiceSettings();
settings.PartitionCount = 1;
settings.ResolvedTransportConnectionString = "MemoryF"; // don't bother with EventHubs for this test
// use a fresh hubname on every run
settings.HubName = $"{TestConstants.TaskHubName}-{Guid.NewGuid()}";
// we don't want to take checkpoints in this test, so we set the following checkpoint triggers unattainably high
settings.MaxNumberBytesBetweenCheckpoints = 1024L * 1024 * 1024 * 1024;
settings.MaxNumberEventsBetweenCheckpoints = 10000000000L;
settings.IdleCheckpointFrequencyMs = (long)TimeSpan.FromDays(1).TotalMilliseconds;
this.settings.PartitionCount = 1;
this.SetCheckpointFrequency(CheckpointFrequency.None);
// set the memory size very small so we can force evictions
settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters()
this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters()
{
StoreLogPageSizeBits = 10, // 1 KB
StoreLogMemorySizeBits = 12, // 4 KB, which means only about 166 entries fit into memory
};
// create a cache monitor
var cacheDebugger = new Faster.CacheDebugger(settings.TestHooks);
var cts = new CancellationTokenSource();
string reportedProblem = null;
settings.TestHooks.OnError += (message) =>
{
this.output?.Invoke($"TESTHOOKS: {message}");
reportedProblem = reportedProblem ?? message;
cts.Cancel();
};
settings.TestHooks.CacheDebugger = cacheDebugger;
// we use the standard hello orchestration from the samples, which calls 5 activities in sequence
var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5);
var activityType = typeof(ScenarioTests.Activities.Hello);
@ -144,17 +171,7 @@ namespace DurableTask.Netherite.Tests
int OrchestrationCount = 100; // requires 200 FASTER key-value pairs so it does not fit into memory
// start the service
var service = new NetheriteOrchestrationService(settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
await orchestrationService.CreateAsync();
await orchestrationService.StartAsync();
var host = (TransportAbstraction.IHost)service;
Assert.Equal(1u, service.NumberPartitions);
var worker = new TaskHubWorker(service);
var client = new TaskHubClient(service);
worker.AddTaskOrchestrations(orchestrationType);
worker.AddTaskActivities(activityType);
await worker.StartAsync();
var (service, client) = await this.StartService(recover: false, orchestrationType, activityType);
// start all orchestrations
{
@ -163,10 +180,10 @@ namespace DurableTask.Netherite.Tests
tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), null);
var timeout = TimeSpan.FromMinutes(3);
var terminationTask = Task.Delay(timeout, cts.Token);
var terminationTask = Task.Delay(timeout, this.cts.Token);
var completionTask = Task.WhenAll(tasks);
var firstTask = await Task.WhenAny(terminationTask, completionTask);
Assert.True(reportedProblem == null, $"CacheDebugger detected problem while starting orchestrations: {reportedProblem}");
Assert.True(this.errorInTestHooks == null, $"while starting orchestrations: {this.errorInTestHooks}");
Assert.True(firstTask != terminationTask, $"timed out after {timeout} while starting orchestrations");
}
@ -217,22 +234,21 @@ namespace DurableTask.Netherite.Tests
PrintUnfinished();
}
cts.Cancel();
this.cts.Cancel();
}
var thread = new Thread(ProgressReportThread);
thread.Name = "ProgressReportThread";
var thread = TrackedThreads.MakeTrackedThread(ProgressReportThread, "ProgressReportThread");
thread.Start();
var terminationTask = Task.Delay(timeout, cts.Token);
var terminationTask = Task.Delay(timeout, this.cts.Token);
var completionTask = Task.WhenAll(tasks);
var firstTask = await Task.WhenAny(terminationTask, completionTask);
Assert.True(reportedProblem == null, $"CacheDebugger detected problem while executing orchestrations: {reportedProblem}");
Assert.True(this.errorInTestHooks == null, $"while executing orchestrations: {this.errorInTestHooks}");
PrintUnfinished();
Assert.True(firstTask != terminationTask, $"timed out after {timeout} while executing orchestrations");
foreach (var line in cacheDebugger.Dump())
foreach (var line in this.cacheDebugger.Dump())
{
this.output?.Invoke(line);
}
@ -240,7 +256,7 @@ namespace DurableTask.Netherite.Tests
catch (Exception e)
{
this.output?.Invoke($"exception thrown while executing orchestrations: {e}");
foreach (var line in cacheDebugger.Dump())
foreach (var line in this.cacheDebugger.Dump())
{
this.output?.Invoke(line);
}
@ -248,10 +264,337 @@ namespace DurableTask.Netherite.Tests
}
// shut down the service
await orchestrationService.StopAsync();
await service.StopAsync();
}
/// <summary>
/// Create a partition and then restore it, and use the size tracker again.
/// </summary>
[Fact]
public async Task CheckSizeTrackerOnRecovery()
{
this.settings.PartitionCount = 1;
this.SetCheckpointFrequency(CheckpointFrequency.None);
// set the memory size very small so we can force evictions
this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters()
{
StoreLogPageSizeBits = 10, // 1 KB
StoreLogMemorySizeBits = 12, // 4 KB, which means only about 166 entries fit into memory
};
// we use the standard hello orchestration from the samples, which calls 5 activities in sequence
var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5);
var activityType = typeof(ScenarioTests.Activities.Hello);
string InstanceId(int i) => $"Orch{i:D5}";
int OrchestrationCount = 100; // requires 200 FASTER key-value pairs so it does not fit into memory
{
// start the service
var (service, client) = await this.StartService(recover: false, orchestrationType, activityType);
// start all orchestrations
{
var tasks = new Task[OrchestrationCount];
for (int i = 0; i < OrchestrationCount; i++)
tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), null);
await Task.WhenAll(tasks);
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}");
}
// wait for all orchestrations to finish executing
{
async Task WaitFor(int i)
{
try
{
await client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10));
}
catch (Exception e)
{
this.output?.Invoke($"Orchestration {InstanceId(i)} failed with {e.GetType()}: {e.Message}");
}
}
var tasks = new Task[OrchestrationCount];
for (int i = 0; i < OrchestrationCount; i++)
tasks[i] = WaitFor(i);
await Task.WhenAll(tasks);
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while executing orchestrations: {this.errorInTestHooks}");
}
this.output?.Invoke("--- test progress: BEFORE SHUTDOWN ------------------------------------");
foreach (var line in this.cacheDebugger.Dump())
{
this.output?.Invoke(line);
}
// shut down the service
await service.StopAsync();
}
{
this.output?.Invoke("--- test progress: BEFORE RECOVERY ------------------------------------");
// recover the service
var (service, client) = await this.StartService(recover: true, orchestrationType, activityType);
this.output?.Invoke("--- test progress: AFTER RECOVERY ------------------------------------");
// query the status of all orchestrations
{
var tasks = new Task[OrchestrationCount];
for (int i = 0; i < OrchestrationCount; i++)
tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10));
await Task.WhenAll(tasks);
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while querying orchestration states: {this.errorInTestHooks}");
}
this.output?.Invoke("--- test progress: AFTER QUERIES ------------------------------------");
foreach (var line in this.cacheDebugger.Dump())
{
this.output?.Invoke(line);
}
// shut down the service
await service.StopAsync();
}
}
/// <summary>
/// Fill memory, then compute size, then reduce page count, and measure size again
/// </summary>
[Fact]
public async Task PipelinedStart()
{
this.settings.PartitionCount = 1;
this.settings.InstanceCacheSizeMB = 2;
this.SetCheckpointFrequency(CheckpointFrequency.Frequent);
var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5);
var activityType = typeof(ScenarioTests.Activities.Hello);
string InstanceId(int i) => $"Orch{i:D5}";
int numOrchestrations = 500;
// start the service
var (service, client) = await this.StartService(recover: false, orchestrationType, activityType);
// start all orchestrations and then get the status of each one
{
var orchestrations = await Enumerable.Range(0, numOrchestrations).ParallelForEachAsync(200, true, (iteration) =>
{
var orchestrationInstanceId = InstanceId(iteration);
return client.CreateOrchestrationInstanceAsync(orchestrationType, orchestrationInstanceId, null);
});
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}");
await Enumerable.Range(0, numOrchestrations).ParallelForEachAsync(200, true, (iteration) =>
{
return client.GetOrchestrationStateAsync(orchestrations[iteration]);
});
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while checking progress of orchestrations: {this.errorInTestHooks}");
}
await service.StopAsync();
}
/// <summary>
/// Repro fail on basic 1000 * hello
/// </summary>
[Fact]
public async Task CheckMemorySize()
{
this.settings.PartitionCount = 1;
this.SetCheckpointFrequency(CheckpointFrequency.None);
var orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn);
var activityType = typeof(ScenarioTests.Activities.Echo);
string InstanceId(int i) => $"Orch{i:D5}";
int OrchestrationCount = 30;
int FanOut = 7;
long historyAndStatusSize = OrchestrationCount * (FanOut * 50000 /* in history */ + 2 * 16000 /* in status */);
// start the service
var (service, client) = await this.StartService(recover: false, orchestrationType, activityType);
// run all orchestrations
{
var tasks = new Task[OrchestrationCount];
for (int i = 0; i < OrchestrationCount; i++)
tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), FanOut);
await Task.WhenAll(tasks);
for (int i = 0; i < OrchestrationCount; i++)
tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(3));
await Task.WhenAll(tasks);
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}");
}
(int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize();
Assert.InRange(memorySize, historyAndStatusSize, 1.05 * historyAndStatusSize);
await service.StopAsync();
}
/// <summary>
/// Fill up memory, then compute size, then reduce page count, and measure size again
/// </summary>
[Fact]
public async Task CheckMemoryReduction()
{
this.settings.PartitionCount = 1;
this.SetCheckpointFrequency(CheckpointFrequency.None);
int pageCountBits = 3;
int pageCount = 1 << pageCountBits;
// set the memory size very small so we can force evictions
this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters()
{
StoreLogPageSizeBits = 9, // 512 B
StoreLogMemorySizeBits = 9 + pageCountBits,
};
var orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn);
var activityType = typeof(ScenarioTests.Activities.Echo);
string InstanceId(int i) => $"Orch{i:D5}";
int OrchestrationCount = 50;
int FanOut = 3;
long historyAndStatusSize = OrchestrationCount * (FanOut * 50000 /* in history */ + 2*16000 /* in status */);
// start the service
var (service, client) = await this.StartService(recover: false, orchestrationType, activityType);
// run all orchestrations
{
var tasks = new Task[OrchestrationCount];
for (int i = 0; i < OrchestrationCount; i++)
tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), FanOut);
await Task.WhenAll(tasks);
for (int i = 0; i < OrchestrationCount; i++)
tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(3));
await Task.WhenAll(tasks);
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}");
}
{
(int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize();
Assert.InRange(numPages, 1, pageCount);
Assert.InRange(memorySize, 0, historyAndStatusSize * 1.1);
}
int emptyPageCount = 0;
int tolerance = 1;
for (int i = 0; i < 4; i++)
{
emptyPageCount++;
this.cacheDebugger.MemoryTracker.SetEmptyPageCount(emptyPageCount);
await Task.Delay(TimeSpan.FromSeconds(20));
(int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize();
Assert.InRange(numPages, 1, pageCount - emptyPageCount + tolerance);
Assert.InRange(memorySize, 0, historyAndStatusSize * 1.1);
}
await service.StopAsync();
}
/// <summary>
/// Run orchestrations to exceed the cache size and check that things are evicted to stay within the target
/// </summary>
[Theory]
[InlineData(false)]
[InlineData(true)]
public async Task CheckMemoryControl(bool useSpaceConsumingOrchestrations)
{
this.settings.PartitionCount = 1;
this.settings.FasterTuningParameters = new BlobManager.FasterTuningParameters()
{
StoreLogPageSizeBits = 10
};
this.SetCheckpointFrequency(CheckpointFrequency.None);
this.settings.TestHooks.CacheDebugger.EnableSizeChecking = false; // our size checker is not entirely accurate in low-memory situationss
Type orchestrationType, activityType;
long SizePerInstance;
object input;
int portionSize;
double uppertolerance;
double lowertolerance;
if (useSpaceConsumingOrchestrations)
{
this.settings.InstanceCacheSizeMB = 4;
orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn);
activityType = typeof(ScenarioTests.Activities.Echo);
int FanOut = 1;
input = FanOut;
SizePerInstance = FanOut * 50000 /* in history */ + 16000 /* in status */;
portionSize = 50;
uppertolerance = 1.1;
lowertolerance = 0;
}
else
{
this.settings.InstanceCacheSizeMB = 2;
orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5);
activityType = typeof(ScenarioTests.Activities.Hello);
SizePerInstance = 3610 /* empiric */;
input = null;
portionSize = 300;
uppertolerance = 1.1;
lowertolerance = 0.9;
}
// start the service
var (service, client) = await this.StartService(recover: false, orchestrationType, activityType);
int logBytesPerInstance = 2 * 40;
long memoryPerPage = ((1 << this.settings.FasterTuningParameters.StoreLogPageSizeBits.Value) / logBytesPerInstance) * SizePerInstance;
double memoryRangeTo = (this.settings.InstanceCacheSizeMB.Value - 1) * 1024 * 1024;
double memoryRangeFrom = (memoryRangeTo - memoryPerPage);
memoryRangeTo = Math.Max(memoryRangeTo, MemoryTracker.MinimumMemoryPages * memoryPerPage);
memoryRangeTo = uppertolerance * memoryRangeTo;
memoryRangeFrom = lowertolerance * memoryRangeFrom;
double pageRangeFrom = Math.Max(MemoryTracker.MinimumMemoryPages, Math.Floor(memoryRangeFrom / memoryPerPage));
double pageRangeTo = Math.Ceiling(memoryRangeTo / memoryPerPage);
async Task AddOrchestrationsAsync(int numOrchestrations)
{
var tasks = new Task<OrchestrationInstance>[numOrchestrations];
for (int i = 0; i < numOrchestrations; i++)
tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, Guid.NewGuid().ToString(), input);
await Task.WhenAll(tasks);
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}");
var tasks2 = new Task<OrchestrationState>[numOrchestrations];
for (int i = 0; i < numOrchestrations; i++)
tasks2[i] = client.WaitForOrchestrationAsync(tasks[i].Result, TimeSpan.FromMinutes(3));
await Task.WhenAll(tasks2);
Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while waiting for orchestrations: {this.errorInTestHooks}");
}
for (int i = 0; i < 4; i++)
{
this.output("memory control ------------------- Add orchestrations");
{
await AddOrchestrationsAsync(portionSize);
this.output("memory control -------- wait for effect");
await Task.Delay(TimeSpan.FromSeconds(10));
this.output("memory control -------- check memory size");
(int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize();
Assert.InRange(numPages, pageRangeFrom, pageRangeTo);
Assert.InRange(memorySize, memoryRangeFrom, memoryRangeTo);
}
}
await service.StopAsync();
}
/// <summary>
/// Test behavior of queries and point queries
@ -259,18 +602,17 @@ namespace DurableTask.Netherite.Tests
[Fact]
public async Task QueriesCopyToTail()
{
var settings = TestConstants.GetNetheriteOrchestrationServiceSettings();
settings.ResolvedTransportConnectionString = "MemoryF";
settings.PartitionCount = 1;
settings.HubName = $"{TestConstants.TaskHubName}-{Guid.NewGuid()}";
var checkpointInjector = settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(settings.TestHooks);
this.settings.PartitionCount = 1;
this.SetCheckpointFrequency(CheckpointFrequency.None);
var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks);
var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn);
var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline);
{
// start the service
var service = new NetheriteOrchestrationService(settings, this.loggerFactory);
var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceClient = (IOrchestrationServiceClient)service;
var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service;
@ -284,28 +626,35 @@ namespace DurableTask.Netherite.Tests
worker.AddTaskOrchestrations(orchestrationType2);
await worker.StartAsync();
int numExtraEntries = 0;
// check that log contains no records
var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Assert.Equal(0 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
// create 100 instances
var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99);
await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40));
var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None);
numExtraEntries += 2;
Assert.Equal(100, instances.Count);
// check that log contains 200 records
var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Assert.Equal(200 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.BeginAddress);
// take a foldover checkpoint
log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null));
Assert.Equal(200 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress);
// read all instances using a query and check that the log did not grow
// (because queries do not copy to tail)
instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None);
Assert.Equal(100, instances.Count);
log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null));
Assert.Equal(200 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress);
// read all instances using point queries and check that the log grew by one record per instance
@ -313,21 +662,22 @@ namespace DurableTask.Netherite.Tests
var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false));
await Task.WhenAll(tasks);
log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Assert.Equal(300 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize);
numExtraEntries += 1;
Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize - 1 * extraLogEntrySize);
// doing the same again has no effect
// (because all instances are already in the mutable section)
tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false));
await Task.WhenAll(tasks);
log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Assert.Equal(300 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize);
Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize - 1 * extraLogEntrySize);
// take a foldover checkpoint
// this moves the readonly section back to the end
log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null));
Assert.Equal(300 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress);
// stop the service
@ -335,26 +685,25 @@ namespace DurableTask.Netherite.Tests
}
}
/// <summary>
/// Test log compaction
/// </summary>
[Fact]
public async Task Compaction()
{
var settings = TestConstants.GetNetheriteOrchestrationServiceSettings();
settings.ResolvedTransportConnectionString = "MemoryF";
settings.PartitionCount = 1;
settings.HubName = $"{TestConstants.TaskHubName}-{Guid.NewGuid()}";
var checkpointInjector = settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(settings.TestHooks);
this.settings.PartitionCount = 1;
this.SetCheckpointFrequency(CheckpointFrequency.None);
var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks);
var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn);
var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline);
long compactUntil = 0;
{
// start the service
var service = new NetheriteOrchestrationService(settings, this.loggerFactory);
var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceClient = (IOrchestrationServiceClient)service;
var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service;
@ -374,12 +723,15 @@ namespace DurableTask.Netherite.Tests
var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None);
Assert.Equal(100, instances.Count);
int numExtraEntries = 1;
// repeat foldover and copy to tail to inflate the log
for (int i = 0; i < 4; i++)
{
// take a foldover checkpoint
var log2 = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null));
Assert.Equal((200 + (100 * i)) * log2.FixedRecordSize, log2.TailAddress - log2.BeginAddress);
numExtraEntries += 1;
Assert.Equal((200 + (100 * i)) * log2.FixedRecordSize + numExtraEntries * extraLogEntrySize, log2.TailAddress - log2.BeginAddress);
Assert.Equal(log2.ReadOnlyAddress, log2.TailAddress);
// read all instances using point queries to force copy to tail
@ -390,13 +742,14 @@ namespace DurableTask.Netherite.Tests
// do log compaction
var log = await checkpointInjector.InjectAsync(log =>
{
compactUntil = 500 * log.FixedRecordSize + log.BeginAddress;
compactUntil = 500 * log.FixedRecordSize + log.BeginAddress + numExtraEntries * extraLogEntrySize;
Assert.Equal(compactUntil, log.SafeReadOnlyAddress);
return (Faster.StoreWorker.CheckpointTrigger.Compaction, compactUntil);
});
// check that the compaction had the desired effect
Assert.Equal(200 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
numExtraEntries = 2;
Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(compactUntil, log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress);
@ -405,7 +758,7 @@ namespace DurableTask.Netherite.Tests
}
{
// recover the service
var service = new NetheriteOrchestrationService(settings, this.loggerFactory);
var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service;
await orchestrationService.CreateAsync();
@ -413,9 +766,11 @@ namespace DurableTask.Netherite.Tests
var host = (TransportAbstraction.IHost)service;
Assert.Equal(1u, service.NumberPartitions);
int numExtraEntries = 2;
// check the log positions
var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Assert.Equal(200 * log.FixedRecordSize, log.TailAddress - log.BeginAddress);
Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress);
Assert.Equal(compactUntil, log.BeginAddress);
Assert.Equal(log.ReadOnlyAddress, log.TailAddress);
@ -427,5 +782,97 @@ namespace DurableTask.Netherite.Tests
await orchestrationService.StopAsync();
}
}
/// <summary>
/// Test log compaction that fails right after compaction
/// </summary>
[Fact]
public async Task CompactThenFail()
{
this.settings.PartitionCount = 1;
this.SetCheckpointFrequency(CheckpointFrequency.None);
var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks);
var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn);
var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline);
long currentTail = 0;
long currentBegin = 0;
long compactUntil = 0;
{
// start the service
var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceClient = (IOrchestrationServiceClient)service;
var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service;
await orchestrationService.CreateAsync();
await orchestrationService.StartAsync();
var host = (TransportAbstraction.IHost)service;
Assert.Equal(1u, service.NumberPartitions);
var worker = new TaskHubWorker(service);
var client = new TaskHubClient(service);
worker.AddTaskOrchestrations(orchestrationType);
worker.AddTaskOrchestrations(orchestrationType2);
await worker.StartAsync();
// create 100 instances
var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99);
await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40));
var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None);
Assert.Equal(100, instances.Count);
int numExtraEntries = 1;
// repeat foldover and copy to tail to inflate the log
for (int i = 0; i < 4; i++)
{
// take a foldover checkpoint
var log2 = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null));
numExtraEntries += 1;
Assert.Equal((200 + (100 * i)) * log2.FixedRecordSize + numExtraEntries * extraLogEntrySize, log2.TailAddress - log2.BeginAddress);
Assert.Equal(log2.ReadOnlyAddress, log2.TailAddress);
currentTail = log2.TailAddress;
currentBegin = log2.BeginAddress;
// read all instances using point queries to force copy to tail
var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false));
await Task.WhenAll(tasks);
}
// do log compaction
var log = await checkpointInjector.InjectAsync(log =>
{
compactUntil = 500 * log.FixedRecordSize + log.BeginAddress + numExtraEntries * extraLogEntrySize;
Assert.Equal(compactUntil, log.SafeReadOnlyAddress);
return (Faster.StoreWorker.CheckpointTrigger.Compaction, compactUntil);
},
injectFailureAfterCompaction:true);
await orchestrationService.StopAsync();
}
{
// recover the service
var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory);
var orchestrationService = (IOrchestrationService)service;
var orchestrationServiceClient = (IOrchestrationServiceClient)service;
await orchestrationService.CreateAsync();
await orchestrationService.StartAsync();
Assert.Equal(this.settings.PartitionCount, (int)service.NumberPartitions);
var worker = new TaskHubWorker(service);
var client = new TaskHubClient(service);
await worker.StartAsync();
// check that begin and tail are the same
var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null));
Debug.Assert(log.BeginAddress == currentBegin);
Debug.Assert(log.TailAddress == currentTail);
}
}
}
}

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

@ -50,7 +50,7 @@ namespace DurableTask.Netherite.Tests
// inject faults with growing success runs until the partition has successfully started
using (this.faultInjector.WithMode(Faster.FaultInjector.InjectionMode.IncrementSuccessRuns, injectDuringStartup: true))
{
fixture = await SingleHostFixture.StartNew(this.settings, true, TimeSpan.FromMinutes(2), (msg) => this.outputHelper.WriteLine(msg));
fixture = await SingleHostFixture.StartNew(this.settings, true, true, null, TimeSpan.FromMinutes(2), (msg) => this.outputHelper.WriteLine(msg));
await this.faultInjector.WaitForStartup(this.settings.PartitionCount, TimeSpan.FromMinutes(2));
}
@ -67,7 +67,7 @@ namespace DurableTask.Netherite.Tests
[Fact]
public async Task InjectHelloCreation()
{
using (var fixture = await SingleHostFixture.StartNew(this.settings, true, TimeSpan.FromMinutes(1), (msg) => this.outputHelper?.WriteLine(msg)))
using (var fixture = await SingleHostFixture.StartNew(this.settings, true, true, null, TimeSpan.FromMinutes(1), (msg) => this.outputHelper?.WriteLine(msg)))
{
await this.faultInjector.WaitForStartup(this.settings.PartitionCount, TimeSpan.FromSeconds(30));
@ -91,7 +91,7 @@ namespace DurableTask.Netherite.Tests
[Fact]
public async Task InjectHelloCompletion()
{
using (var fixture = await SingleHostFixture.StartNew(this.settings, true, TimeSpan.FromMinutes(1), (msg) => this.outputHelper.WriteLine(msg)))
using (var fixture = await SingleHostFixture.StartNew(this.settings, true, true, null, TimeSpan.FromMinutes(1), (msg) => this.outputHelper.WriteLine(msg)))
{
// do not start injecting until all partitions have started
await this.faultInjector.WaitForStartup(this.settings.PartitionCount, TimeSpan.FromSeconds(30));

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

@ -25,6 +25,7 @@ namespace DurableTask.Netherite.Tests
{
readonly SingleHostFixture fixture;
readonly TestOrchestrationHost host;
readonly Action<string> output;
ITestOutputHelper outputHelper;
public QueryTests(SingleHostFixture fixture, ITestOutputHelper outputHelper)
@ -32,18 +33,28 @@ namespace DurableTask.Netherite.Tests
this.fixture = fixture;
this.host = fixture.Host;
this.outputHelper = outputHelper;
this.output = (string message) => this.outputHelper?.WriteLine(message);
this.output($"Running pre-test operations on {fixture.GetType().Name}.");
this.fixture.SetOutput((message) => this.outputHelper?.WriteLine(message));
this.fixture.SetOutput(this.output);
Assert.False(fixture.HasError(out var error), $"could not start test because of preceding test failure: {error}");
// purge all instances prior to each test
if (! this.host.PurgeAllAsync().Wait(TimeSpan.FromMinutes(3)))
{
throw new TimeoutException("timed out while purging instances before starting test");
}
this.output($"Completed pre-test operations on {fixture.GetType().Name}.");
}
public void Dispose()
{
this.output($"Running post-test operations on {this.fixture.GetType().Name}.");
Assert.False(this.fixture.HasError(out var error), $"detected test failure: {error}");
// purge all instances after each test
// this helps to catch "bad states" (e.g. hung workers) caused by the tests
if (!this.host.PurgeAllAsync().Wait(TimeSpan.FromMinutes(3)))
@ -53,6 +64,9 @@ namespace DurableTask.Netherite.Tests
Assert.Null(this.fixture.TestHooksError);
this.fixture.DumpCacheDebugger();
this.output($"Completed post-test operations on {this.fixture.GetType().Name}.");
this.outputHelper = null;
}
@ -123,7 +137,7 @@ namespace DurableTask.Netherite.Tests
}
// Make sure the client and instance are still running and didn't complete early (or fail).
var status = await client.GetStatusAsync();
var status = await client.GetStateAsync();
Assert.NotNull(status);
Assert.Contains(status.OrchestrationStatus, inProgressStatus);
await assertCounts(1, 0);

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

@ -28,19 +28,30 @@ namespace DurableTask.Netherite.Tests
{
readonly SingleHostFixture fixture;
readonly TestOrchestrationHost host;
readonly Action<string> output;
ITestOutputHelper outputHelper;
public ScenarioTests(SingleHostFixture fixture, ITestOutputHelper outputHelper)
{
this.outputHelper = outputHelper;
this.output = (string message) => this.outputHelper?.WriteLine(message);
this.output($"Running pre-test operations on {fixture.GetType().Name}.");
this.fixture = fixture;
this.host = fixture.Host;
fixture.SetOutput(this.output);
Assert.False(fixture.HasError(out var error), $"could not start test because of preceding test failure: {error}");
fixture.SetOutput((string message) => this.outputHelper?.WriteLine(message));
this.output($"Completed pre-test operations on {fixture.GetType().Name}.");
}
public void Dispose()
{
this.output($"Running post-test operations on {this.fixture.GetType().Name}.");
Assert.False(this.fixture.HasError(out var error), $"detected test failure: {error}");
// purge all instances after each test
// this helps to catch "bad states" (e.g. hung workers) caused by the tests
if (!this.host.PurgeAllAsync().Wait(TimeSpan.FromMinutes(3)))
@ -49,6 +60,9 @@ namespace DurableTask.Netherite.Tests
}
Assert.Null(this.fixture.TestHooksError);
this.fixture.DumpCacheDebugger();
this.output($"Completed post-test operations on {this.fixture.GetType().Name}.");
this.outputHelper = null;
}
@ -233,8 +247,8 @@ namespace DurableTask.Netherite.Tests
var client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.Counter), initialValue);
// Need to wait for the instance to start before sending events to it.
// TODO: This requirement may not be ideal and should be revisited.
await client.WaitForStartupAsync(TimeSpan.FromSeconds(10));
var state = await client.WaitForStartupAsync(TimeSpan.FromSeconds(30));
Assert.NotNull(state);
// Perform some operations
await client.RaiseEventAsync(Orchestrations.Counter.OpEventName, Orchestrations.Counter.OpIncrement);
@ -245,21 +259,22 @@ namespace DurableTask.Netherite.Tests
await Task.Delay(2000);
// Make sure it's still running and didn't complete early (or fail).
var status = await client.GetStatusAsync();
state = await client.GetStateAsync();
Assert.NotNull(state);
Assert.True(
status?.OrchestrationStatus == OrchestrationStatus.Running ||
status?.OrchestrationStatus == OrchestrationStatus.ContinuedAsNew);
state.OrchestrationStatus == OrchestrationStatus.Running ||
state.OrchestrationStatus == OrchestrationStatus.ContinuedAsNew);
// The end message will cause the actor to complete itself.
await client.RaiseEventAsync(Orchestrations.Counter.OpEventName, Orchestrations.Counter.OpEnd);
status = await client.WaitForCompletionAsync(TimeSpan.FromSeconds(10));
state = await client.WaitForCompletionAsync(TimeSpan.FromSeconds(10));
Assert.Equal(OrchestrationStatus.Completed, status?.OrchestrationStatus);
Assert.Equal(3, JToken.Parse(status?.Output));
Assert.Equal(OrchestrationStatus.Completed, state?.OrchestrationStatus);
Assert.Equal(3, JToken.Parse(state?.Output));
// When using ContinueAsNew, the original input is discarded and replaced with the most recent state.
Assert.NotEqual(initialValue, JToken.Parse(status?.Input));
Assert.NotEqual(initialValue, JToken.Parse(state?.Input));
}
/// <summary>
@ -870,11 +885,11 @@ namespace DurableTask.Netherite.Tests
var clientStartingIn10Seconds = await this.host.StartOrchestrationAsync(typeof(Orchestrations.DelayedCurrentTimeActivity), "Delayed Current Time!", startAt: expectedStartTime);
var clientStartingNow = await this.host.StartOrchestrationAsync(typeof(Orchestrations.DelayedCurrentTimeActivity), "Delayed Current Time!");
var statusStartingIn10Seconds = await clientStartingIn10Seconds.GetStatusAsync();
var statusStartingIn10Seconds = await clientStartingIn10Seconds.GetStateAsync();
Assert.NotNull(statusStartingIn10Seconds.ScheduledStartTime);
Assert.Equal(expectedStartTime, statusStartingIn10Seconds.ScheduledStartTime);
var statusStartingNow = await clientStartingNow.GetStatusAsync();
var statusStartingNow = await clientStartingNow.GetStateAsync();
Assert.Null(statusStartingNow.ScheduledStartTime);
await Task.WhenAll(

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

@ -3,6 +3,7 @@
namespace DurableTask.Netherite.Tests
{
using DurableTask.Netherite.Faster;
using Microsoft.Extensions.Logging;
using System;
using System.Collections.Generic;
@ -18,18 +19,20 @@ namespace DurableTask.Netherite.Tests
{
readonly TestTraceListener traceListener;
readonly XunitLoggerProvider loggerProvider;
readonly CacheDebugger cacheDebugger;
internal TestOrchestrationHost Host { get; private set; }
internal ILoggerFactory LoggerFactory { get; private set; }
internal string TestHooksError { get; private set; }
public SingleHostFixture()
: this(TestConstants.GetNetheriteOrchestrationServiceSettings(), true, null)
: this(TestConstants.GetNetheriteOrchestrationServiceSettings(), true, true, null, null)
{
this.Host.StartAsync().Wait();
}
SingleHostFixture(NetheriteOrchestrationServiceSettings settings, bool useReplayChecker, Action<string> output)
SingleHostFixture(NetheriteOrchestrationServiceSettings settings, bool useCacheDebugger, bool useReplayChecker, int? restrictMemory, Action<string> output)
{
this.LoggerFactory = new LoggerFactory();
this.loggerProvider = new XunitLoggerProvider();
@ -37,22 +40,30 @@ namespace DurableTask.Netherite.Tests
this.traceListener = new TestTraceListener() { Output = output };
Trace.Listeners.Add(this.traceListener);
TestConstants.ValidateEnvironment();
string timestamp = DateTime.UtcNow.ToString("yyyyMMdd-HHmmss-fffffff");
settings.HubName = $"SingleHostFixture-{timestamp}";
settings.PartitionManagement = PartitionManagementOptions.EventProcessorHost;
settings.InstanceCacheSizeMB = restrictMemory;
if (useCacheDebugger)
{
this.cacheDebugger = settings.TestHooks.CacheDebugger = new Faster.CacheDebugger(settings.TestHooks);
}
if (useReplayChecker)
{
settings.TestHooks.ReplayChecker = new Faster.ReplayChecker(settings.TestHooks);
}
settings.TestHooks.OnError += (message) =>
{
System.Diagnostics.Trace.WriteLine($"TESTHOOKS: {message}");
Trace.WriteLine($"TESTHOOKS: {message}");
this.TestHooksError ??= message;
};
// start the host
this.Host = new TestOrchestrationHost(settings, this.LoggerFactory);
}
public static async Task<SingleHostFixture> StartNew(NetheriteOrchestrationServiceSettings settings, bool useReplayChecker, TimeSpan timeout, Action<string> output)
public static async Task<SingleHostFixture> StartNew(NetheriteOrchestrationServiceSettings settings, bool useCacheDebugger, bool useReplayChecker, int? restrictMemory, TimeSpan timeout, Action<string> output)
{
var fixture = new SingleHostFixture(settings, useReplayChecker, output);
var fixture = new SingleHostFixture(settings, useCacheDebugger, useReplayChecker, restrictMemory, output);
var startupTask = fixture.Host.StartAsync();
timeout = TestOrchestrationClient.AdjustTimeout(timeout);
var timeoutTask = Task.Delay(timeout);
@ -65,6 +76,14 @@ namespace DurableTask.Netherite.Tests
return fixture;
}
public void DumpCacheDebugger()
{
foreach (var line in this.cacheDebugger.Dump())
{
Trace.WriteLine(line);
}
}
public void Dispose()
{
this.Host.StopAsync(false).Wait();
@ -72,6 +91,13 @@ namespace DurableTask.Netherite.Tests
Trace.Listeners.Remove(this.traceListener);
}
public bool HasError(out string error)
{
error = this.TestHooksError;
return error != null;
}
// called before a new test, to route output to the test output
public void SetOutput(Action<string> output)
{
this.traceListener.Output = output;
@ -82,7 +108,16 @@ namespace DurableTask.Netherite.Tests
{
public Action<string> Output { get; set; }
public override void Write(string message) { }
public override void WriteLine(string message) { this.Output?.Invoke($"{DateTime.Now:o} {message}"); }
public override void WriteLine(string message) {
try
{
this.Output?.Invoke($"{DateTime.Now:o} {message}");
}
catch(System.InvalidOperationException e) when (e.Message.Contains("There is no currently active test"))
{
// This exception is sometimes thrown by xunit when reusing the same fixture for multiple tests.
}
}
}
}
}

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

@ -11,7 +11,7 @@ namespace DurableTask.Netherite.Tests
{
public const string StorageConnectionName ="AzureWebJobsStorage";
public const string EventHubsConnectionName ="EventHubsConnection";
public const string TaskHubName ="test-taskhub";
public const string DefaultTaskHubName ="test-taskhub";
public static void ValidateEnvironment()
{
@ -31,7 +31,7 @@ namespace DurableTask.Netherite.Tests
{
StorageConnectionName = StorageConnectionName,
EventHubsConnectionName = EventHubsConnectionName,
HubName = TaskHubName,
HubName = DefaultTaskHubName,
TransportLogLevelLimit = LogLevel.Trace,
StorageLogLevelLimit = LogLevel.Trace,
LogLevelLimit = LogLevel.Trace,
@ -65,7 +65,5 @@ namespace DurableTask.Netherite.Tests
internal static TestOrchestrationHost GetTestOrchestrationHost(ILoggerFactory loggerFactory)
=> new TestOrchestrationHost(GetNetheriteOrchestrationServiceSettings(), loggerFactory);
public static bool DeleteStorageBeforeRunningTests => true; // set to false for testing log-based recovery
}
}

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

@ -117,7 +117,7 @@ namespace DurableTask.Netherite.Tests
Stopwatch sw = Stopwatch.StartNew();
do
{
OrchestrationState state = await this.GetStatusAsync();
OrchestrationState state = await this.GetStateAsync();
if (state != null && state.OrchestrationStatus != OrchestrationStatus.Pending)
{
Trace.TraceInformation($"TestProgress: Started {state.Name} id={state.OrchestrationInstance.InstanceId} after ~{sw.ElapsedMilliseconds}ms. Status = {state.OrchestrationStatus}.");
@ -142,7 +142,7 @@ namespace DurableTask.Netherite.Tests
return null;
}
public async Task<OrchestrationState> GetStatusAsync()
public async Task<OrchestrationState> GetStateAsync()
{
OrchestrationState state = await this.client.GetOrchestrationStateAsync(this.instanceId);

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

@ -27,11 +27,6 @@ namespace DurableTask.Netherite.Tests
this.orchestrationService = new Netherite.NetheriteOrchestrationService(settings, loggerFactory);
var orchestrationService = (IOrchestrationService)this.orchestrationService;
if (TestConstants.DeleteStorageBeforeRunningTests)
{
orchestrationService.DeleteAsync().GetAwaiter().GetResult();
}
orchestrationService.CreateAsync(false).GetAwaiter().GetResult();
this.settings = settings;

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

@ -12,13 +12,13 @@ namespace PerformanceTests
public class LauncherEntity
{
public void Launch((string orchestrationName, int numberOrchestrations, int offset, string input) input)
public void Launch((string orchestrationName, string prefix, int numberOrchestrations, int offset, string input) input)
{
// start all the orchestrations
for (int iteration = 0; iteration < input.numberOrchestrations; iteration++)
{
int globalIteration = iteration + input.offset;
var orchestrationInstanceId = ManyOrchestrations.InstanceId(globalIteration);
var orchestrationInstanceId = ManyOrchestrations.InstanceId(input.prefix, globalIteration);
Entity.Current.StartNewOrchestration(input.orchestrationName, input.input ?? globalIteration.ToString(), orchestrationInstanceId);
};
}

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

@ -21,26 +21,23 @@ namespace PerformanceTests
using DurableTask.Core.Stats;
/// <summary>
/// Http triggers for starting, awaiting, counting, or purging large numbers of orchestration instances
/// operations for starting, awaiting, counting, or purging large numbers of orchestration instances
///
/// Example invocations:
/// curl https://.../start -d HelloSequence.1000 launch 1000 HelloSequence instances, from the http trigger
/// curl https://.../start -d HelloSequence.10000.200 launch 10000 HelloSequence instances, in portions of 200, from launcher entities
/// curl https://.../start -d XYZ.10000.200/abc launch 10000 XYZ instances, in portions of 200, with inputs abc
/// curl https://.../await -d 1000 waits for the 1000 instances to complete
/// curl https://.../count -d 1000 check the status of the 1000 instances and reports the (last completed - first started) time range
/// curl https://.../purge -d 1000 purges the 1000 instances
/// curl https://.../query issues a query to check the status of all orchestrations
/// Example curl invocations:
/// .../start -d HelloSequence.1000 launch 1000 HelloSequence instances, from the http trigger
/// .../start -d HelloSequence.10000.200 launch 10000 HelloSequence instances, in portions of 200, from launcher entities
/// .../start -d XYZ.10000.200/abc launch 10000 XYZ instances, in portions of 200, with inputs abc
/// .../await -d 1000 waits for the 1000 instances to complete
/// .../count -d 1000 check the status of the 1000 instances and reports the (last completed - first started) time range
/// .../purge -d 1000 purges the 1000 instances
/// .../query issues a query to check the status of all orchestrations
///
/// </summary>
public static class ManyOrchestrations
{
[FunctionName(nameof(Start))]
public static async Task<IActionResult> Start(
[HttpTrigger(AuthorizationLevel.Anonymous, "post")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
{
public static async Task<IActionResult> Start(HttpRequest req, IDurableClient client, ILogger log, string prefix)
{
try
{
string requestBody = await new StreamReader(req.Body).ReadToEndAsync();
@ -88,7 +85,7 @@ namespace PerformanceTests
// start all the orchestrations
await Enumerable.Range(0, numberOrchestrations).ParallelForEachAsync(200, true, (iteration) =>
{
var orchestrationInstanceId = InstanceId(iteration);
var orchestrationInstanceId = InstanceId(prefix, iteration);
log.LogInformation($"starting {orchestrationInstanceId}");
return client.StartNewAsync(orchestrationName, orchestrationInstanceId, input ?? iteration.ToString());
});
@ -111,7 +108,7 @@ namespace PerformanceTests
{
int portion = Math.Min(portionSize.Value, (numberOrchestrations - pos));
var entityId = new EntityId(nameof(LauncherEntity), $"launcher{launcher / 100:D6}!{launcher % 100:D2}");
tasks.Add(client.SignalEntityAsync(entityId, nameof(LauncherEntity.Launch), (orchestrationName, portion, pos, input)));
tasks.Add(client.SignalEntityAsync(entityId, nameof(LauncherEntity.Launch), (orchestrationName, prefix, portion, pos, input)));
pos += portion;
launcher++;
}
@ -126,11 +123,7 @@ namespace PerformanceTests
}
}
[FunctionName(nameof(Await))]
public static async Task<IActionResult> Await(
[HttpTrigger(AuthorizationLevel.Anonymous, "post")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
public static async Task<IActionResult> Await(HttpRequest req, IDurableClient client, ILogger log, string prefix)
{
string requestBody = await new StreamReader(req.Body).ReadToEndAsync();
int numberOrchestrations = int.Parse(requestBody);
@ -151,7 +144,7 @@ namespace PerformanceTests
// wait for all the orchestrations
await Enumerable.Range(0, numberOrchestrations).ParallelForEachAsync(200, true, async (iteration) =>
{
var orchestrationInstanceId = InstanceId(iteration);
var orchestrationInstanceId = InstanceId(prefix, iteration);
IActionResult response = await client.WaitForCompletionOrCreateCheckStatusResponseAsync(req, orchestrationInstanceId, deadline - DateTime.UtcNow);
if (response is ObjectResult objectResult
@ -175,11 +168,7 @@ namespace PerformanceTests
}
}
[FunctionName(nameof(Count))]
public static async Task<IActionResult> Count(
[HttpTrigger(AuthorizationLevel.Anonymous, "post", Route = "count")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
public static async Task<IActionResult> Count(HttpRequest req, IDurableClient client, ILogger log, string prefix)
{
try
{
@ -189,7 +178,7 @@ namespace PerformanceTests
var queryCondition = new OrchestrationStatusQueryCondition()
{
InstanceIdPrefix = "Orch",
InstanceIdPrefix = prefix,
};
int completed = 0;
@ -211,7 +200,7 @@ namespace PerformanceTests
log.LogWarning($"Checking the status of {numberOrchestrations} orchestration instances...");
await Enumerable.Range(0, numberOrchestrations).ParallelForEachAsync(200, true, async (iteration) =>
{
var orchestrationInstanceId = InstanceId(iteration);
var orchestrationInstanceId = InstanceId(prefix, iteration);
var status = await client.GetStatusAsync(orchestrationInstanceId);
lock (lockForUpdate)
@ -270,17 +259,13 @@ namespace PerformanceTests
}
}
[FunctionName(nameof(Query))]
public static async Task<IActionResult> Query(
[HttpTrigger(AuthorizationLevel.Anonymous, "get")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
public static async Task<IActionResult> Query(HttpRequest req, IDurableClient client, ILogger log, string prefix)
{
try
{
var queryCondition = new OrchestrationStatusQueryCondition()
{
InstanceIdPrefix = "Orch",
InstanceIdPrefix = prefix,
};
int completed = 0;
@ -345,11 +330,7 @@ namespace PerformanceTests
}
}
[FunctionName(nameof(Purge))]
public static async Task<IActionResult> Purge(
[HttpTrigger(AuthorizationLevel.Anonymous, "post")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
public static async Task<IActionResult> Purge(HttpRequest req, IDurableClient client, ILogger log, string prefix)
{
string requestBody = await new StreamReader(req.Body).ReadToEndAsync();
int numberOrchestrations = int.Parse(requestBody);
@ -370,7 +351,7 @@ namespace PerformanceTests
// start all the orchestrations
await Enumerable.Range(0, numberOrchestrations).ParallelForEachAsync(200, true, async (iteration) =>
{
var orchestrationInstanceId = InstanceId(iteration);
var orchestrationInstanceId = InstanceId(prefix, iteration);
var response = await client.PurgeInstanceHistoryAsync(orchestrationInstanceId);
Interlocked.Add(ref deleted, response.InstancesDeleted);
@ -390,16 +371,16 @@ namespace PerformanceTests
// we can use this to run on a subset of the available partitions
static readonly int? restrictedPlacement = null;
public static string InstanceId(int index)
public static string InstanceId(string prefix, int index)
{
if (restrictedPlacement == null)
{
return $"Orch{index:X5}";
return $"{prefix}{index:X5}";
}
else
{
return $"Orch{index:X5}!{(index % restrictedPlacement):D2}";
return $"{prefix}{index:X5}!{(index % restrictedPlacement):D2}";
}
}
}
}
}

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

@ -0,0 +1,118 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
namespace PerformanceTests
{
using System;
using System.IO;
using System.Threading.Tasks;
using Microsoft.AspNetCore.Mvc;
using Microsoft.Azure.WebJobs;
using Microsoft.Azure.WebJobs.Extensions.Http;
using Microsoft.AspNetCore.Http;
using Microsoft.Extensions.Logging;
using Newtonsoft.Json;
using System.Collections.Generic;
using Microsoft.Azure.WebJobs.Extensions.DurableTask;
using System.Linq;
using System.Collections.Concurrent;
using System.Threading;
using System.Net.Http;
using DurableTask.Core.Stats;
/// <summary>
/// Http triggers for operations on large numbers of orchestration instances
///
/// Example curl invocations:
/// curl https://.../start ... launch instances with prefix "Orch"
/// curl https://.../many/abc/start .. launch instances with prefix "abc"
///
/// </summary>
public static class ManyOrchestrationsHttp
{
[FunctionName(nameof(Start))]
public static Task<IActionResult> Start(
[HttpTrigger(AuthorizationLevel.Anonymous, "post")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
{
return ManyOrchestrations.Start(req, client, log, "Orch");
}
[FunctionName(nameof(Await))]
public static Task<IActionResult> Await(
[HttpTrigger(AuthorizationLevel.Anonymous, "post")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
{
return ManyOrchestrations.Await(req, client, log, "Orch");
}
[FunctionName(nameof(Count))]
public static Task<IActionResult> Count(
[HttpTrigger(AuthorizationLevel.Anonymous, "post", Route = "count")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
{
return ManyOrchestrations.Count(req, client, log, "Orch");
}
[FunctionName(nameof(Query))]
public static Task<IActionResult> Query(
[HttpTrigger(AuthorizationLevel.Anonymous, "get")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
{
return ManyOrchestrations.Query(req, client, log, "Orch");
}
[FunctionName(nameof(Purge))]
public static Task<IActionResult> Purge(
[HttpTrigger(AuthorizationLevel.Anonymous, "post")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log)
{
return ManyOrchestrations.Purge(req, client, log, "Orch");
}
[FunctionName(nameof(ManyOrchestrationsPost))]
public static async Task<IActionResult> ManyOrchestrationsPost(
[HttpTrigger(AuthorizationLevel.Anonymous, "post", Route = "many/{prefix}/{action}")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log,
string prefix,
string action)
{
switch (action)
{
case "start":
return await ManyOrchestrations.Start(req, client, log, prefix);
case "purge":
return await ManyOrchestrations.Purge(req, client, log, prefix);
case "count":
return await ManyOrchestrations.Count(req, client, log, prefix);
case "await":
return await ManyOrchestrations.Await(req, client, log, prefix);
default:
return new NotFoundObjectResult($"no such action: POST ${action}");
}
}
[FunctionName(nameof(ManyOrchestrationsGet))]
public static async Task<IActionResult> ManyOrchestrationsGet(
[HttpTrigger(AuthorizationLevel.Anonymous, "get", Route = "many/{prefix}/{action}")] HttpRequest req,
[DurableClient] IDurableClient client,
ILogger log,
string prefix,
string action)
{
switch (action)
{
case "query":
return await ManyOrchestrations.Query(req, client, log, prefix);
default:
return new NotFoundObjectResult($"no such action: GET ${action}");
}
}
}
}

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

@ -64,6 +64,9 @@
// set this to false to force replay on each orchestration step
"CacheOrchestrationCursors": true,
// set this to control the max size of the orchestration instance cache
// "InstanceCacheSizeMB": "2",
// set this to true to use the PSF support in Faster. Will soon be obsolete.
"UsePSFQueries": "false",
@ -77,6 +80,8 @@
//"FaultInjectionProbability": 250,
// set this to true to attach replay checker
//"AttachReplayChecker": true,
// set this to true to attach replay checker
//"AttachCacheDebugger": true,
// set this to "Scripted" to control the scenario with a partition script
// or to "ClientOnly" to run only the client