From 7e33bb8a41e5925acb2692a3c77421a3bbc470f3 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Tue, 27 Jan 2015 17:11:56 +0100 Subject: [PATCH 01/13] Reimpleted the mapping between Future handles and FdbFuture classes [BROKEN] - Future handles are no longer stored in safe handles, to prevent the GC from destroying the futures behind our back - FdbFutureContext is the base class of all "contexts" that can have futures (ie: clusters, transactions, ...) - There is a static dictionary that stores all active contexts (first routing step) - Each context has also its own dictionary that stores all active futures in this context (second rounting step) - future callback parameter is the concatenation of the context ID and the future ID and is used for the two-step routing - Multi-futures (FdbFutureArray) use refcounting and only fire once the last handle has fired --- FoundationDB.Client/FdbWatch.cs | 6 +- .../FoundationDB.Client.csproj | 2 + FoundationDB.Client/Native/FdbFuture.cs | 412 +-------------- FoundationDB.Client/Native/FdbFutureArray.cs | 324 ++++-------- .../Native/FdbFutureContext.cs | 477 ++++++++++++++++++ FoundationDB.Client/Native/FdbFutureSingle.cs | 257 +++------- FoundationDB.Client/Native/FdbNative.cs | 133 ++--- .../Native/FdbNativeCluster.cs | 80 +-- .../Native/FdbNativeDatabase.cs | 48 +- .../Native/FdbNativeTransaction.cs | 232 ++++----- .../Native/Handles/FutureHandle.cs | 4 + FoundationDB.Client/Native/IFdbFuture.cs | 53 ++ 12 files changed, 952 insertions(+), 1076 deletions(-) create mode 100644 FoundationDB.Client/Native/FdbFutureContext.cs create mode 100644 FoundationDB.Client/Native/IFdbFuture.cs diff --git a/FoundationDB.Client/FdbWatch.cs b/FoundationDB.Client/FdbWatch.cs index f447b5b36..4938aed65 100644 --- a/FoundationDB.Client/FdbWatch.cs +++ b/FoundationDB.Client/FdbWatch.cs @@ -88,10 +88,13 @@ public TaskAwaiter GetAwaiter() if (m_future != null) { +#if REFACTORING_IN_PROGRESS if (m_future.HasFlag(FdbFuture.Flags.DISPOSED)) { throw new ObjectDisposedException("Cannot await a watch that has already been disposed"); } + +#endif return m_future.Task.GetAwaiter(); } throw new InvalidOperationException("Cannot await an empty watch"); @@ -111,7 +114,8 @@ public void Dispose() { if (m_future != null) { - m_future.Dispose(); + //TODO: what should be do? (=> cancel the future?) + //m_future.Dispose(); } } diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 82e12c10f..7fdf19767 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -73,6 +73,8 @@ + + diff --git a/FoundationDB.Client/Native/FdbFuture.cs b/FoundationDB.Client/Native/FdbFuture.cs index ad4106c00..0827e7adc 100644 --- a/FoundationDB.Client/Native/FdbFuture.cs +++ b/FoundationDB.Client/Native/FdbFuture.cs @@ -31,382 +31,39 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Native { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; - using System.Collections.Concurrent; - using System.Collections.Generic; using System.Diagnostics; using System.Runtime.CompilerServices; - using System.Threading; using System.Threading.Tasks; - /// Helper class to create FDBFutures - internal static class FdbFuture - { - - public static class Flags - { - /// The future has completed (either success or failure) - public const int COMPLETED = 1; - - /// A completion/failure/cancellation has been posted on the thread pool - public const int HAS_POSTED_ASYNC_COMPLETION = 2; - - /// The future has been cancelled from an external source (manually, or via then CancellationTokeb) - public const int CANCELLED = 4; - - /// The resources allocated by this future have been released - public const int MEMORY_RELEASED = 8; - - /// The future has been constructed, and is listening for the callbacks - public const int READY = 64; - - /// Dispose has been called - public const int DISPOSED = 128; - } - - /// Create a new from an FDBFuture* pointer - /// Type of the result of the task - /// FDBFuture* pointer - /// Func that will be called to get the result once the future completes (and did not fail) - /// Optional cancellation token that can be used to cancel the future - /// Object that tracks the execution of the FDBFuture handle - [NotNull] - public static FdbFutureSingle FromHandle([NotNull] FutureHandle handle, [NotNull] Func selector, CancellationToken cancellationToken) - { - return new FdbFutureSingle(handle, selector, cancellationToken); - } - - /// Create a new from an array of FDBFuture* pointers - /// Type of the items of the arrayreturn by the task - /// Array of FDBFuture* pointers - /// Func that will be called for each future that complete (and did not fail) - /// Optional cancellation token that can be used to cancel the future - /// Object that tracks the execution of all the FDBFuture handles - [NotNull] - public static FdbFutureArray FromHandleArray([NotNull] FutureHandle[] handles, [NotNull] Func selector, CancellationToken cancellationToken) - { - return new FdbFutureArray(handles, selector, cancellationToken); - } - - /// Wrap a FDBFuture* pointer into a - /// Type of the result of the task - /// FDBFuture* pointer - /// Lambda that will be called once the future completes sucessfully, to extract the result from the future handle. - /// Optional cancellation token that can be used to cancel the future - /// Task that will either return the result of the continuation lambda, or an exception - public static Task CreateTaskFromHandle([NotNull] FutureHandle handle, [NotNull] Func continuation, CancellationToken cancellationToken) - { - return new FdbFutureSingle(handle, continuation, cancellationToken).Task; - } - - /// Wrap multiple handles into a single that returns an array of T - /// Type of the result of the task - /// Array of FDBFuture* pointers - /// Lambda that will be called once for each future that completes sucessfully, to extract the result from the future handle. - /// Optional cancellation token that can be used to cancel the future - /// Task that will either return all the results of the continuation lambdas, or an exception - /// If at least one future fails, the whole task will fail. - public static Task CreateTaskFromHandleArray([NotNull] FutureHandle[] handles, [NotNull] Func continuation, CancellationToken cancellationToken) - { - // Special case, because FdbFutureArray does not support empty arrays - //TODO: technically, there is no reason why FdbFutureArray would not accept an empty array. We should simplify this by handling the case in the ctor (we are already allocating something anyway...) - if (handles.Length == 0) return Task.FromResult(new T[0]); - - return new FdbFutureArray(handles, continuation, cancellationToken).Task; - } - - } - /// Base class for all FDBFuture wrappers /// Type of the Task's result - [DebuggerDisplay("Flags={m_flags}, State={this.Task.Status}")] - internal abstract class FdbFuture : TaskCompletionSource, IDisposable + [DebuggerDisplay("Label={Label}, Cookie={Cookie}, State={Task.Status}")] + internal abstract class FdbFuture : TaskCompletionSource, IFdbFuture { #region Private Members... - /// Flags of the future (bit field of FLAG_xxx values) - private int m_flags; - - /// Future key in the callback dictionary - protected IntPtr m_key; - - /// Optionnal registration on the parent Cancellation Token - /// Is only valid if FLAG_HAS_CTR is set - protected CancellationTokenRegistration m_ctr; - - #endregion - - #region State Management... - - internal bool HasFlag(int flag) - { - return (Volatile.Read(ref m_flags) & flag) == flag; - } - - internal bool HasAnyFlags(int flags) - { - return (Volatile.Read(ref m_flags) & flags) != 0; - } - - protected void SetFlag(int flag) - { - var flags = m_flags; - Thread.MemoryBarrier(); - m_flags = flags | flag; - } - - protected bool TrySetFlag(int flag) - { - var wait = new SpinWait(); - while (true) - { - var flags = Volatile.Read(ref m_flags); - if ((flags & flag) != 0) - { - return false; - } - if (Interlocked.CompareExchange(ref m_flags, flags | flag, flags) == flags) - { - return true; - } - wait.SpinOnce(); - } - } - - protected bool TryCleanup() - { - // We try to cleanup the future handle as soon as possible, meaning as soon as we have the result, or an error, or a cancellation - - if (TrySetFlag(FdbFuture.Flags.COMPLETED)) - { - DoCleanup(); - return true; - } - return false; - } - - private void DoCleanup() - { - try - { - // unsubscribe from the parent cancellation token if there was one - UnregisterCancellationRegistration(); - - // ensure that the task always complete ! - // note: always defer the completion on the threadpool, because we don't want to dead lock here (we can be called by Dispose) - if (!this.Task.IsCompleted && TrySetFlag(FdbFuture.Flags.HAS_POSTED_ASYNC_COMPLETION)) - { - PostCancellationOnThreadPool(this); - } - - // The only surviving value after this would be a Task and an optional WorkItem on the ThreadPool that will signal it... - } - finally - { - CloseHandles(); - } - } - - /// Close all the handles managed by this future - protected abstract void CloseHandles(); - - /// Cancel all the handles managed by this future - protected abstract void CancelHandles(); - - /// Release all memory allocated by this future - protected abstract void ReleaseMemory(); - - /// Set the result of this future - /// Result of the future - /// If true, called from the network thread callback and will defer the operation on the ThreadPool. If false, may run the continuations inline. - protected void SetResult(T result, bool fromCallback) - { - if (!fromCallback) - { - this.TrySetResult(result); - } - else if (TrySetFlag(FdbFuture.Flags.HAS_POSTED_ASYNC_COMPLETION)) - { - PostCompletionOnThreadPool(this, result); - } - } - - /// Fault the future's Task - /// Error that will be the result of the task - /// If true, called from the network thread callback and will defer the operation on the ThreadPool. If false, may run the continuations inline. - protected void SetFaulted(Exception e, bool fromCallback) - { - if (!fromCallback) - { - this.TrySetException(e); - } - else if (TrySetFlag(FdbFuture.Flags.HAS_POSTED_ASYNC_COMPLETION)) - { - PostFailureOnThreadPool(this, e); - } - } - - /// Fault the future's Task - /// Error that will be the result of the task - /// If true, called from the network thread callback and will defer the operation on the ThreadPool. If false, may run the continuations inline. - protected void SetFaulted(IEnumerable errors, bool fromCallback) - { - if (!fromCallback) - { - this.TrySetException(errors); - } - else if (TrySetFlag(FdbFuture.Flags.HAS_POSTED_ASYNC_COMPLETION)) - { - PostFailureOnThreadPool(this, errors); - } - } - - /// Cancel the future's Task - /// If true, called from the network thread callback and will defer the operation on the ThreadPool. If false, may run the continuations inline. - protected void SetCanceled(bool fromCallback) - { - if (!fromCallback) - { - this.TrySetCanceled(); - } - else if (TrySetFlag(FdbFuture.Flags.HAS_POSTED_ASYNC_COMPLETION)) - { - PostCancellationOnThreadPool(this); - } - } - - /// Defer setting the result of a TaskCompletionSource on the ThreadPool - private static void PostCompletionOnThreadPool(TaskCompletionSource future, T result) - { - ThreadPool.UnsafeQueueUserWorkItem( - (_state) => - { - var prms = (Tuple, T>)_state; - prms.Item1.TrySetResult(prms.Item2); - }, - Tuple.Create(future, result) - ); - } + ///// Optionnal registration on the parent Cancellation Token + ///// Is only valid if FLAG_HAS_CTR is set + //protected CancellationTokenRegistration m_ctr; - /// Defer failing a TaskCompletionSource on the ThreadPool - private static void PostFailureOnThreadPool(TaskCompletionSource future, Exception error) + protected FdbFuture(IntPtr cookie, string label) { - ThreadPool.UnsafeQueueUserWorkItem( - (_state) => - { - var prms = (Tuple, Exception>)_state; - prms.Item1.TrySetException(prms.Item2); - }, - Tuple.Create(future, error) - ); - } - - /// Defer failing a TaskCompletionSource on the ThreadPool - private static void PostFailureOnThreadPool(TaskCompletionSource future, IEnumerable errors) - { - ThreadPool.UnsafeQueueUserWorkItem( - (_state) => - { - var prms = (Tuple, IEnumerable>)_state; - prms.Item1.TrySetException(prms.Item2); - }, - Tuple.Create(future, errors) - ); + this.Cookie = cookie; + this.Label = label; } - /// Defer cancelling a TaskCompletionSource on the ThreadPool - private static void PostCancellationOnThreadPool(TaskCompletionSource future) - { - ThreadPool.UnsafeQueueUserWorkItem( - (_state) => ((TaskCompletionSource)_state).TrySetCanceled(), - future - ); - } - - #endregion - - #region Callbacks... - - /// List of all pending futures that have not yet completed - private static readonly ConcurrentDictionary> s_futures = new ConcurrentDictionary>(); - - /// Internal counter to generated a unique parameter value for each futures - private static long s_futureCounter; - - /// Register a future in the callback context and return the corresponding callback parameter - /// Future instance - /// Parameter that can be passed to FutureSetCallback and that uniquely identify this future. - /// The caller MUST call ClearCallbackHandler to ensure that the future instance is removed from the list - internal static IntPtr RegisterCallback([NotNull] FdbFuture future) - { - Contract.Requires(future != null); - - // generate a new unique id for this future, that will be use to lookup the future instance in the callback handler - long id = Interlocked.Increment(ref s_futureCounter); - var prm = new IntPtr(id); // note: we assume that we can only run in 64-bit mode, so it is safe to cast a long into an IntPtr - // critical region - try { } - finally - { - Volatile.Write(ref future.m_key, prm); -#if DEBUG_FUTURES - Contract.Assert(!s_futures.ContainsKey(prm)); -#endif - s_futures[prm] = future; - Interlocked.Increment(ref DebugCounters.CallbackHandlesTotal); - Interlocked.Increment(ref DebugCounters.CallbackHandles); - } - return prm; - } - - /// Remove a future from the callback handler dictionary - /// Future that has just completed, or is being destroyed - internal static void UnregisterCallback([NotNull] FdbFuture future) - { - Contract.Requires(future != null); - - // critical region - try - { } - finally - { - var key = Interlocked.Exchange(ref future.m_key, IntPtr.Zero); - if (key != IntPtr.Zero) - { - FdbFuture _; - if (s_futures.TryRemove(key, out _)) - { - Interlocked.Decrement(ref DebugCounters.CallbackHandles); - } - } - } - } + public IntPtr Cookie { get; private set; } - internal static FdbFuture GetFutureFromCallbackParameter(IntPtr parameter) - { - FdbFuture future; - if (s_futures.TryGetValue(parameter, out future)) - { - if (future != null && Volatile.Read(ref future.m_key) == parameter) - { - return future; - } -#if DEBUG_FUTURES - // If you breakpoint here, that means that a future callback fired but was not able to find a matching registration - // => either the FdbFuture was incorrectly disposed, or there is some problem in the callback dictionary - if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); -#endif - } - return null; - } + public string Label { get; private set; } #endregion #region Cancellation... +#if REFACTORED + protected void RegisterForCancellation(CancellationToken cancellationToken) { //note: if the token is already cancelled, the callback handler will run inline and any exception would bubble up here @@ -437,8 +94,14 @@ private static void CancellationHandler(object state) } } +#endif + #endregion + public abstract bool Visit(IntPtr handle); + + public abstract void OnFired(); + /// Return true if the future has completed (successfully or not) public bool IsReady { @@ -454,6 +117,8 @@ public TaskAwaiter GetAwaiter() /// Try to abort the task (if it is still running) public void Cancel() { + throw new NotImplementedException("FIXME: Future Cancellation!"); +#if REFACTORED if (HasAnyFlags(FdbFuture.Flags.DISPOSED | FdbFuture.Flags.COMPLETED | FdbFuture.Flags.CANCELLED)) { return; @@ -475,42 +140,7 @@ public void Cancel() TryCleanup(); } } - } - - /// Free memory allocated by this future after it has completed. - /// This method provides no benefit to most application code, and should only be called when attempting to write thread-safe custom layers. - public void Clear() - { - if (HasFlag(FdbFuture.Flags.DISPOSED)) - { - return; - } - - if (!this.Task.IsCompleted) - { - throw new InvalidOperationException("Cannot release memory allocated by a future that has not yet completed"); - } - - if (TrySetFlag(FdbFuture.Flags.MEMORY_RELEASED)) - { - ReleaseMemory(); - } - } - - public void Dispose() - { - if (TrySetFlag(FdbFuture.Flags.DISPOSED)) - { - try - { - TryCleanup(); - } - finally - { - if (Volatile.Read(ref m_key) != IntPtr.Zero) UnregisterCallback(this); - } - } - GC.SuppressFinalize(this); +#endif } } diff --git a/FoundationDB.Client/Native/FdbFutureArray.cs b/FoundationDB.Client/Native/FdbFutureArray.cs index 0a7989478..bffcf5c0a 100644 --- a/FoundationDB.Client/Native/FdbFutureArray.cs +++ b/FoundationDB.Client/Native/FdbFutureArray.cs @@ -30,7 +30,6 @@ namespace FoundationDB.Client.Native { using JetBrains.Annotations; using System; - using System.Collections.Generic; using System.Diagnostics; using System.Threading; @@ -38,286 +37,155 @@ namespace FoundationDB.Client.Native /// Type of result internal sealed class FdbFutureArray : FdbFuture { - // Wraps several FDBFuture* handles and return all the results at once #region Private Members... - /// Value of the 'FDBFuture*' - private readonly FutureHandle[] m_handles; + private IntPtr[] m_handles; - /// Counter of callbacks that still need to fire. private int m_pending; - /// Lambda used to extract the result of this FDBFuture - private readonly Func m_resultSelector; + private readonly Func m_resultSelector; - #endregion + private readonly object m_state; - #region Constructors... + #endregion - internal FdbFutureArray([NotNull] FutureHandle[] handles, [NotNull] Func selector, CancellationToken cancellationToken) + internal FdbFutureArray([NotNull] IntPtr[] handles, [NotNull] Func selector, object state, IntPtr cookie, string label) + : base(cookie, label) { - if (handles == null) throw new ArgumentNullException("handles"); - if (handles.Length == 0) throw new ArgumentException("Handle array cannot be empty", "handles"); - if (selector == null) throw new ArgumentNullException("selector"); - m_handles = handles; + m_pending = handles.Length; m_resultSelector = selector; - - bool abortAllHandles = false; - - try - { - if (cancellationToken.IsCancellationRequested) - { // already cancelled, we must abort everything - - SetFlag(FdbFuture.Flags.COMPLETED); - abortAllHandles = true; - m_resultSelector = null; - this.TrySetCanceled(); - return; - } - - // add this instance to the list of pending futures - var prm = RegisterCallback(this); - - foreach (var handle in handles) - { - - if (FdbNative.FutureIsReady(handle)) - { // this handle is already done - continue; - } - - Interlocked.Increment(ref m_pending); - - // register the callback handler - var err = FdbNative.FutureSetCallback(handle, CallbackHandler, prm); - if (Fdb.Failed(err)) - { // uhoh - Debug.WriteLine("Failed to set callback for Future<" + typeof(T).Name + "> 0x" + handle.Handle.ToString("x") + " !!!"); - throw Fdb.MapToException(err); - } - } - - // allow the callbacks to handle completion - TrySetFlag(FdbFuture.Flags.READY); - - if (Volatile.Read(ref m_pending) == 0) - { // all callbacks have already fired (or all handles were already completed) - UnregisterCallback(this); - HandleCompletion(fromCallback: false); - m_resultSelector = null; - abortAllHandles = true; - SetFlag(FdbFuture.Flags.COMPLETED); - } - else if (cancellationToken.CanBeCanceled) - { // register for cancellation (if needed) - RegisterForCancellation(cancellationToken); - } - } - catch - { - // this is bad news, since we are in the constructor, we need to clear everything - SetFlag(FdbFuture.Flags.DISPOSED); - - UnregisterCancellationRegistration(); - - UnregisterCallback(this); - - abortAllHandles = true; - - // this is technically not needed, but just to be safe... - this.TrySetCanceled(); - - throw; - } - finally - { - if (abortAllHandles) - { - CloseHandles(handles); - } - } - GC.KeepAlive(this); + m_state = state; } - #endregion - - protected override void CloseHandles() + public override bool Visit(IntPtr handle) { - CloseHandles(m_handles); + return 0 == Interlocked.Decrement(ref m_pending); } - protected override void CancelHandles() - { - CancelHandles(m_handles); - } + private const int CATEGORY_SUCCESS = 0; + private const int CATEGORY_RETRYABLE = 1; + private const int CATEGORY_CANCELLED = 2; + private const int CATEGORY_FAILURE = 3; - protected override void ReleaseMemory() + private static int ClassifyErrorSeverity(FdbError error) { - var handles = m_handles; - if (handles != null) + switch (error) { - foreach (var handle in handles) - { - if (handle != null && !handle.IsClosed && !handle.IsInvalid) - { - //REVIEW: there is a possibility of a race condition with Dispoe() that could potentially call FutureDestroy(handle) at the same time (not verified) - FdbNative.FutureReleaseMemory(handle); - } - } - } - } + case FdbError.Success: + return CATEGORY_SUCCESS; - private static void CloseHandles(FutureHandle[] handles) - { - if (handles != null) - { - foreach (var handle in handles) - { - if (handle != null) - { - //note: Dispose() will be a no-op if already called - handle.Dispose(); - } - } - } - } + case FdbError.PastVersion: + case FdbError.FutureVersion: + case FdbError.TimedOut: + case FdbError.TooManyWatches: + return CATEGORY_RETRYABLE; - private static void CancelHandles(FutureHandle[] handles) - { - if (handles != null) - { - foreach (var handle in handles) - { - if (handle != null && !handle.IsClosed && !handle.IsInvalid) - { - //REVIEW: there is a possibility of a race condition with Dispoe() that could potentially call FutureDestroy(handle) at the same time (not verified) - FdbNative.FutureCancel(handle); - } - } + case FdbError.OperationCancelled: + case FdbError.TransactionCancelled: + return CATEGORY_CANCELLED; + + default: + return CATEGORY_FAILURE; } } - /// Cached delegate of the future completion callback handler - private static readonly FdbNative.FdbFutureCallback CallbackHandler = FutureCompletionCallback; - - /// Handler called when a FDBFuture becomes ready - /// Handle on the future that became ready - /// Paramter to the callback (unused) - private static void FutureCompletionCallback(IntPtr futureHandle, IntPtr parameter) + public override void OnFired() { -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + ">.Callback(0x" + futureHandle.ToString("x") + ", " + parameter.ToString("x") + ") has fired on thread #" + Thread.CurrentThread.ManagedThreadId.ToString()); -#endif + var handles = Interlocked.Exchange(ref m_handles, null); + if (handles == null) return; // already disposed? - var future = (FdbFutureArray)GetFutureFromCallbackParameter(parameter); + Debug.WriteLine("Future{0}<{1}[]>.OnFired({2})", this.Label, typeof (T).Name, handles.Length); - if (future != null && Interlocked.Decrement(ref future.m_pending) == 0) - { // the last future handle has fired, we can proceed to read all the results + //README: + // - This callback will fire either from the ThreadPool (async ops) or inline form the ctor of the future (non-async ops, or ops that where served from some cache). + // - The method *MUST* dispose the future handle before returning, and *SHOULD* do so before signaling the task. + // => This is because continuations may run inline, and start new futures from there, while we still have our original future handle opened. - if (future.HasFlag(FdbFuture.Flags.READY)) + try + { + T[] results = new T[handles.Length]; + FdbError code = FdbError.Success; + int severity = 0; + Exception error = null; + try { - UnregisterCallback(future); - try - { - future.HandleCompletion(fromCallback: true); + if (this.Task.IsCompleted) + { // task has already been handled by someone else + return; } - catch(Exception) - { - //TODO ? - } - } - // else, the ctor will handle that - } - } - - /// Update the Task with the state of a ready Future - /// If true, the method is called from the network thread and must defer the continuations from the Thread Pool - /// True if we got a result, or false in case of error (or invalid state) - private void HandleCompletion(bool fromCallback) - { - if (HasAnyFlags(FdbFuture.Flags.DISPOSED | FdbFuture.Flags.COMPLETED)) - { - return; - } -#if DEBUG_FUTURES - Debug.WriteLine("FutureArray<" + typeof(T).Name + ">.Callback(...) handling completion on thread #" + Thread.CurrentThread.ManagedThreadId.ToString()); -#endif - - try - { - UnregisterCancellationRegistration(); + for (int i = 0; i < results.Length; i++) + { + var handle = handles[i]; + var err = FdbNative.FutureGetError(handle); + if (err == FdbError.Success) + { + if (code != FdbError.Success) + { // there's been at least one error before, so there is no point in computing the result, it would be discarded anyway + continue; + } - List errors = null; - bool cancellation = false; - var selector = m_resultSelector; + try + { + results[i] = m_resultSelector(handle, m_state); + } + catch (AccessViolationException e) + { // trouble in paradise! - var results = selector != null ? new T[m_handles.Length] : null; + Debug.WriteLine("EPIC FAIL: " + e.ToString()); - for (int i = 0; i < m_handles.Length; i++) - { - var handle = m_handles[i]; + // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) + if (Debugger.IsAttached) Debugger.Break(); - if (handle != null && !handle.IsClosed && !handle.IsInvalid) - { - FdbError err = FdbNative.FutureGetError(handle); - if (Fdb.Failed(err)) - { // it failed... - if (err != FdbError.OperationCancelled) - { // get the exception from the error code - var ex = Fdb.MapToException(err); - (errors ?? (errors = new List())).Add(ex); + Environment.FailFast("FIXME: FDB done goofed!", e); } - else + catch (Exception e) { - cancellation = true; + Debug.WriteLine("FAIL: " + e.ToString()); + code = FdbError.InternalError; + error = e; break; } } - else - { // it succeeded... - // try to get the result... - if (selector != null) - { - //note: result selector will execute from network thread, but this should be our own code that only calls into some fdb_future_get_XXXX(), which should be safe... - results[i] = selector(handle); + else if (code != err) + { + int cur = ClassifyErrorSeverity(err); + if (cur > severity) + { // error is more serious than before + severity = cur; + code = err; } } } } + finally + { + foreach (var handle in handles) + { + if (handle != IntPtr.Zero) FdbNative.FutureDestroy(handle); + } + } - if (cancellation) - { // the transaction has been cancelled - SetCanceled(fromCallback); + if (code == FdbError.Success) + { + TrySetResult(results); } - else if (errors != null) - { // there was at least one error - SetFaulted(errors, fromCallback); + else if (code == FdbError.OperationCancelled || code == FdbError.TransactionCancelled) + { + TrySetCanceled(); } else - { // success - SetResult(results, fromCallback); - } - - } - catch (Exception e) - { // something went wrong - if (e is ThreadAbortException) { - SetCanceled(fromCallback); - throw; + TrySetException(error ?? Fdb.MapToException(code)); } - SetFaulted(e, fromCallback); } - finally - { - TryCleanup(); + catch (Exception e) + { // we must not blow up the TP or the parent, so make sure to propagate all exceptions to the task + TrySetException(e); } } - } -} +} \ No newline at end of file diff --git a/FoundationDB.Client/Native/FdbFutureContext.cs b/FoundationDB.Client/Native/FdbFutureContext.cs new file mode 100644 index 000000000..2dba49c7a --- /dev/null +++ b/FoundationDB.Client/Native/FdbFutureContext.cs @@ -0,0 +1,477 @@ +#region BSD Licence +/* Copyright (c) 2013-2014, Doxense SAS +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of Doxense nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ +#endregion + +// enable this to capture the stacktrace of the ctor, when troubleshooting leaked transaction handles +#undef CAPTURE_STACKTRACES + +using System.IO.IsolatedStorage; + +namespace FoundationDB.Client.Native +{ + using FoundationDB.Client.Core; + using FoundationDB.Client.Utils; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Runtime.CompilerServices; + using System.Threading; + using System.Threading.Tasks; + + internal class FdbFutureContext : IDisposable + { + + #region Private Constants... + + private const int FUTURE_COOKIE_SIZE = 32; + + private const int FUTURE_COOKIE_SHIFT = 0; + + private const ulong FUTURE_COOKIE_MASK = (1UL << FUTURE_COOKIE_SIZE) - 1; + + private const int CONTEXT_COOKIE_SIZE = 32; + + private const ulong CONTEXT_COOKIE_MASK = (1UL << CONTEXT_COOKIE_SIZE) - 1; + + private const int CONTEXT_COOKIE_SHIFT = FUTURE_COOKIE_SIZE; + + #endregion + + #region Static Stuff.... + + /// Counter used to generate the cookie values for each unique context + private static int s_globalCookieCounter; + + private static readonly Dictionary s_contexts = new Dictionary(); + + private static IntPtr MakeCallbackCookie(uint contextId, uint futureId) + { + ulong cookie = (contextId & CONTEXT_COOKIE_MASK) << CONTEXT_COOKIE_SHIFT; + cookie |= (futureId & FUTURE_COOKIE_MASK) << FUTURE_COOKIE_SHIFT; + return new IntPtr((long)cookie); + } + + private static readonly FdbNative.FdbFutureCallback GlobalCallback = FutureCallbackHandler; + + private static void FutureCallbackHandler(IntPtr handle, IntPtr cookie) + { + // cookie is the value that will help us find the corresponding context (upper 32 bits) and future within this context (lower 32 bits) that matches with this future handle. + + //note: this callback can be called either: + // - from the thread that is constructing the Future, if the future was already completed (called inline) + // - from the network thread, when the future completed asynchronously + + Debug.WriteLine("FutureCallbackHandler(0x{0}, {1:X8} | {2:X8}) called", handle.ToString("X"), cookie.ToInt64() >> 32, cookie.ToInt64() & uint.MaxValue); + + bool deferred = false; + try + { + + uint contextId = (uint) (((ulong) cookie.ToInt64() >> CONTEXT_COOKIE_SHIFT) & CONTEXT_COOKIE_MASK); + + FdbFutureContext context; + lock (s_contexts) // there will only be contentions on this lock if other a lot of threads are creating new contexts (ie: new transactions) + { + s_contexts.TryGetValue(contextId, out context); + } + + if (context != null) + { + Contract.Assert(context.m_contextId == contextId); + deferred = context.OnFired(handle, cookie); + } + } + finally + { + if (!deferred) FdbNative.FutureDestroy(handle); + } + } + + #endregion + + + /// Cookie for this context + /// Makes the 32-bits upper bits of the future callback parameter + private readonly uint m_contextId = (uint) Interlocked.Increment(ref s_globalCookieCounter); + + /// Counter used to generated the cookie for all futures created from this context + private int m_localCookieCounter; + + /// Dictionary used to store all the pending Futures for this context + /// All methods should take a lock on this instance before manipulating the state + private readonly Dictionary m_futures = new Dictionary(); + +#if CAPTURE_STACKTRACES + private StackTrace m_stackTrace; +#endif + + #region Constructors... + + protected FdbFutureContext() + { + lock (s_contexts) + { + s_contexts[m_contextId] = this; + } +#if CAPTURE_STACKTRACES + m_stackTrace = new StackTrace(); +#endif + } + + //REVIEW: do we really need a destructor ? The handle is a SafeHandle, and will take care of itself... + ~FdbFutureContext() + { +#if CAPTURE_STACKTRACES + Trace.WriteLine("A transaction handle (" + m_handle + ", " + m_payloadBytes + " bytes written) was leaked by " + m_stackTrace); +#endif +#if DEBUG + // If you break here, that means that a native transaction handler was leaked by a FdbTransaction instance (or that the transaction instance was leaked) + if (Debugger.IsAttached) Debugger.Break(); +#endif + Dispose(false); + } + + #endregion + + #region IDisposable... + + public void Dispose() + { + Dispose(true); + GC.SuppressFinalize(this); + } + + protected virtual void Dispose(bool disposing) + { + //Debug.WriteLine("Disposified from " + new StackTrace()); + if (disposing) + { + lock (s_contexts) + { + Debug.WriteLine("Disposed context {0}#{1} with {2} pending future(s) ({3} total)", this.GetType().Name, m_contextId, m_futures.Count, m_localCookieCounter); + s_contexts.Remove(m_contextId); + foreach (var ctx in s_contexts) + { + Debug.WriteLine("- {0}#{1} : {2} ({3})", ctx.Value.GetType().Name, ctx.Key, ctx.Value.m_futures.Count, ctx.Value.m_localCookieCounter); + } + } + } + } + + #endregion + + /// A callback has fire for a future handled by this context + /// + /// + /// If this flag is set to true, then the caller will NOT destroy the future. + private bool OnFired(IntPtr handle, IntPtr cookie) + { + IFdbFuture future; + lock (m_futures) + { + m_futures.TryGetValue(cookie, out future); + } + + if (future != null && future.Cookie == cookie) + { + if (future.Visit(handle)) + { // future is ready to process all the results + ThreadPool.UnsafeQueueUserWorkItem( + (state) => + { + ((IFdbFuture)state).OnFired(); + //TODO: if it fails, maybe we should remove it from m_futures? + }, + future + ); + } + // else: expecting more handles + + // handles will be destroyed when the future completes + return true; + } + + return false; + } + + /// Add a new future handle to this context + /// + /// Handle of the newly created future + /// Flag set to true if the future must be disposed by the caller (in case of error), or false if the future will be disposed by some other thread. + /// Method called when the future completes successfully + /// TODO: remove this? + /// Type of future (name of the caller) + /// + protected Task RegisterFuture( + IntPtr handle, + ref bool mustDispose, + Func selector, + object state, + CancellationToken ct, + string label + ) + { + FdbFutureSingle future = null; + IntPtr cookie = IntPtr.Zero; + try + { + uint futureId = (uint)Interlocked.Increment(ref m_localCookieCounter); + cookie = MakeCallbackCookie(m_contextId, futureId); + + future = new FdbFutureSingle(handle, selector, state, cookie, label); + + if (FdbNative.FutureIsReady(handle)) + { // the result is already computed + Debug.WriteLine("Future.{0} 0x{1} already completed!", label, handle.ToString("X")); + mustDispose = false; + future.OnFired(); + return future.Task; + } + + lock (m_futures) + { + //TODO: marke the future as "registered" (must unreg when it fires?) + m_futures[cookie] = future; + } + + var err = FdbNative.FutureSetCallback(handle, GlobalCallback, cookie); + if (!Fdb.Success(err)) + { + throw Fdb.MapToException(err); + } + mustDispose = false; + return future.Task; + } + catch (Exception e) + { + if (future != null) + { + future.TrySetException(e); + if (cookie != IntPtr.Zero) + { + lock (m_futures) + { + m_futures.Remove(cookie); + } + } + return future.Task; + } + throw; + } + } + + /// Add a new future handle to this context + /// + /// Handles of the newly created future + /// Flag set to true if the future must be disposed by the caller (in case of error), or false if the future will be disposed by some other thread. + /// Method called when the future completes successfully + /// TODO: remove this? + /// Type of future (name of the caller) + /// + protected Task RegisterFutures( + IntPtr[] handles, + ref bool mustDispose, + Func selector, + object state, + CancellationToken ct, + string label + ) + { + FdbFutureArray future = null; + IntPtr cookie = IntPtr.Zero; + try + { + uint futureId = (uint)Interlocked.Increment(ref m_localCookieCounter); + cookie = MakeCallbackCookie(m_contextId, futureId); + + // make a copy because we may diverge from the caller if we partially fail to register the callbacks below + var tmp = new IntPtr[handles.Length]; + handles.CopyTo(tmp, 0); + future = new FdbFutureArray(tmp, selector, state, cookie, label); + + //TODO: we could check if all handles are already completed/failed? + + lock (m_futures) + { + //TODO: mark the future as "registered" (must unreg when it fires?) + m_futures[cookie] = future; + } + + for (int i = 0; i < handles.Length; i++) + { + var err = FdbNative.FutureSetCallback(handles[i], GlobalCallback, cookie); + if (Fdb.Success(err)) + { + handles[i] = IntPtr.Zero; + } + else + { + // mute this future + lock (m_futures) + { + m_futures.Remove(cookie); + //TODO: mark the future as "unregistered" + for (int j = i + 1; j < handles.Length; j++) + { + tmp[j] = IntPtr.Zero; + } + } + + throw Fdb.MapToException(err); + } + } + mustDispose = false; + return future.Task; + } + catch (Exception e) + { + if (future != null) + { + future.TrySetException(e); + if (cookie != IntPtr.Zero) + { + lock (m_futures) + { + m_futures.Remove(cookie); + } + } + return future.Task; + } + throw; + } + } + + } + + internal class FdbFutureContext : FdbFutureContext + where THandle : FdbSafeHandle + { + + protected readonly THandle m_handle; + + protected FdbFutureContext([NotNull] THandle handle) + { + if (handle == null) throw new ArgumentNullException("handle"); + m_handle = handle; + } + + public THandle Handle { [NotNull] get { return m_handle; } } + + protected override void Dispose(bool disposing) + { + try + { + base.Dispose(disposing); + } + finally + { + if (disposing) + { + lock (this.Handle) + { + if (!this.Handle.IsClosed) this.Handle.Dispose(); + } + } + } + } + + /// Start a new async operation + /// Result of the operation + /// Argument passed to the generator + /// Lambda called to produce the future handle + /// Argument passed to . It will not be used after the handle has been constructed + /// Lambda called once the future completes (successfully) + /// State object passed to . It will be stored in the future has long as it is active. + /// Optional cancellation token used to cancel the task from an external source. + /// Optional label, used for logging and troubleshooting purpose (by default the name of the caller) + /// + protected Task StartNewFuture( + Func generator, + TArg argument, + Func selector, + object state, + CancellationToken ct, + [CallerMemberName] string label = null + ) + { + + bool mustDispose = true; + IntPtr h = IntPtr.Zero; + try + { + lock (this.Handle) + { + if (this.Handle.IsClosed) throw new ObjectDisposedException(this.GetType().Name); + h = generator(m_handle, argument); + } + return RegisterFuture(h, ref mustDispose, selector, state, ct, label); + } + finally + { + if (mustDispose && h != IntPtr.Zero) + { + FdbNative.FutureDestroy(h); + } + } + } + + protected Task StartNewFutures( + int count, + Action generator, + TArg arg, + Func selector, + object state, + CancellationToken ct, + [CallerMemberName] string label = null + + ) + { + bool mustDispose = true; + var handles = new IntPtr[count]; + try + { + lock (this.Handle) + { + if (this.Handle.IsClosed) throw new ObjectDisposedException(this.GetType().Name); + generator(m_handle, arg, handles); + } + return RegisterFutures(handles, ref mustDispose, selector, state, ct, label); + } + catch + { + foreach (var future in handles) + { + if (future != IntPtr.Zero) FdbNative.FutureDestroy(future); + } + throw; + } + } + } + +} diff --git a/FoundationDB.Client/Native/FdbFutureSingle.cs b/FoundationDB.Client/Native/FdbFutureSingle.cs index 48877f1ee..0bb0ff4ee 100644 --- a/FoundationDB.Client/Native/FdbFutureSingle.cs +++ b/FoundationDB.Client/Native/FdbFutureSingle.cs @@ -28,11 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #undef DEBUG_FUTURES +using System.Diagnostics; + namespace FoundationDB.Client.Native { + using FoundationDB.Client.Utils; using JetBrains.Annotations; using System; - using System.Diagnostics; + using System.Runtime.ExceptionServices; using System.Threading; /// FDBFuture wrapper @@ -42,226 +45,106 @@ internal sealed class FdbFutureSingle : FdbFuture #region Private Members... /// Value of the 'FDBFuture*' - private readonly FutureHandle m_handle; + private IntPtr m_handle; /// Lambda used to extract the result of this FDBFuture - private readonly Func m_resultSelector; + private readonly Func m_resultSelector; - #endregion + private readonly object m_state; - #region Constructors... + #endregion - internal FdbFutureSingle([NotNull] FutureHandle handle, [NotNull] Func selector, CancellationToken cancellationToken) + internal FdbFutureSingle(IntPtr handle, [NotNull] Func selector, object state, IntPtr cookie, string label) + : base(cookie, label) { - if (handle == null) throw new ArgumentNullException("handle"); + if (handle == IntPtr.Zero) throw new ArgumentException("Invalid future handle", "handle"); if (selector == null) throw new ArgumentNullException("selector"); m_handle = handle; m_resultSelector = selector; - - try - { - - if (handle.IsInvalid) - { // it's dead, Jim ! - SetFlag(FdbFuture.Flags.COMPLETED); - m_resultSelector = null; - return; - } - - if (FdbNative.FutureIsReady(handle)) - { // either got a value or an error -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + "> 0x" + handle.Handle.ToString("x") + " was already ready"); -#endif - HandleCompletion(fromCallback: false); -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + "> 0x" + handle.Handle.ToString("x") + " completed inline"); -#endif - return; - } - - // register for cancellation (if needed) - if (cancellationToken.CanBeCanceled) - { - if (cancellationToken.IsCancellationRequested) - { // we have already been cancelled - -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + "> 0x" + handle.Handle.ToString("x") + " will complete later"); -#endif - - // Abort the future and simulate a Canceled task - SetFlag(FdbFuture.Flags.COMPLETED); - // note: we don't need to call fdb_future_cancel because fdb_future_destroy will take care of everything - handle.Dispose(); - // also, don't keep a reference on the callback because it won't be needed - m_resultSelector = null; - this.TrySetCanceled(); - return; - } - - // token still active - RegisterForCancellation(cancellationToken); - } - -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + "> 0x" + handle.Handle.ToString("x") + " will complete later"); -#endif - - TrySetFlag(FdbFuture.Flags.READY); - - // add this instance to the list of pending futures - var prm = RegisterCallback(this); - - // register the callback handler - var err = FdbNative.FutureSetCallback(handle, CallbackHandler, prm); - if (Fdb.Failed(err)) - { // uhoh -#if DEBUG_FUTURES - Debug.WriteLine("Failed to set callback for Future<" + typeof(T).Name + "> 0x" + handle.Handle.ToString("x") + " !!!"); -#endif - throw Fdb.MapToException(err); - } - } - catch - { - // this is bad news, since we are in the constructor, we need to clear everything - SetFlag(FdbFuture.Flags.DISPOSED); - UnregisterCancellationRegistration(); - UnregisterCallback(this); - - // kill the future handle - m_handle.Dispose(); - - // this is technically not needed, but just to be safe... - this.TrySetCanceled(); - - throw; - } - GC.KeepAlive(this); + m_state = state; } - #endregion - - /// Cached delegate of the future completion callback handler - private static readonly FdbNative.FdbFutureCallback CallbackHandler = FutureCompletionCallback; - - /// Handler called when a FDBFuture becomes ready - /// Handle on the future that became ready - /// Paramter to the callback (unused) - private static void FutureCompletionCallback(IntPtr futureHandle, IntPtr parameter) + public override bool Visit(IntPtr handle) { -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + ">.Callback(0x" + futureHandle.ToString("x") + ", " + parameter.ToString("x") + ") has fired on thread #" + Thread.CurrentThread.ManagedThreadId.ToString()); -#endif - - var future = (FdbFutureSingle)GetFutureFromCallbackParameter(parameter); - if (future != null) - { - UnregisterCallback(future); - future.HandleCompletion(fromCallback: true); - } + Contract.Requires(handle == m_handle); + return true; } - /// Update the Task with the state of a ready Future - /// If true, we are called from the network thread - /// True if we got a result, or false in case of error (or invalid state) - private void HandleCompletion(bool fromCallback) + [HandleProcessCorruptedStateExceptions] // to be able to handle Access Violations and terminate the process + public override void OnFired() { - // note: if fromCallback is true, we are running on the network thread - // this means that we have to signal the TCS from the threadpool, if not continuations on the task may run inline. - // this is very frequent when we are called with await, or ContinueWith(..., TaskContinuationOptions.ExecuteSynchronously) + Debug.WriteLine("Future{0}<{1}>.OnFired(0x{2})", this.Label, typeof(T).Name, m_handle.ToString("X8")); - if (HasAnyFlags(FdbFuture.Flags.DISPOSED | FdbFuture.Flags.COMPLETED)) - { - return; - } + var handle = Interlocked.Exchange(ref m_handle, IntPtr.Zero); + if (handle == IntPtr.Zero) return; // already disposed? + + //README: + // - This callback will fire either from the ThreadPool (async ops) or inline form the ctor of the future (non-async ops, or ops that where served from some cache). + // - The method *MUST* dispose the future handle before returning, and *SHOULD* do so before signaling the task. + // => This is because continuations may run inline, and start new futures from there, while we still have our original future handle opened. -#if DEBUG_FUTURES - var sw = Stopwatch.StartNew(); -#endif try { - var handle = m_handle; - if (handle != null && !handle.IsClosed && !handle.IsInvalid) + T result = default(T); + FdbError code; + Exception error = null; + try { - UnregisterCancellationRegistration(); + if (this.Task.IsCompleted) + { // task has already been handled by someone else + return; + } + + code = FdbNative.FutureGetError(handle); + if (code == FdbError.Success) + { + try + { + result = m_resultSelector(handle, m_state); + } + catch (AccessViolationException e) + { // trouble in paradise! + + Debug.WriteLine("EPIC FAIL: " + e.ToString()); + + // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) + if (Debugger.IsAttached) Debugger.Break(); - FdbError err = FdbNative.FutureGetError(handle); - if (Fdb.Failed(err)) - { // it failed... -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + "> has FAILED: " + err); -#endif - if (err != FdbError.OperationCancelled) - { // get the exception from the error code - var ex = Fdb.MapToException(err); - SetFaulted(ex, fromCallback); - return; + Environment.FailFast("FIXME: FDB done goofed!", e); } - //else: will be handle below - } - else - { // it succeeded... - // try to get the result... -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + "> has completed successfully"); -#endif - var selector = m_resultSelector; - if (selector != null) + catch (Exception e) { - //note: result selector will execute from network thread, but this should be our own code that only calls into some fdb_future_get_XXXX(), which should be safe... - var result = selector(handle); - SetResult(result, fromCallback); - return; + Debug.WriteLine("FAIL: " + e.ToString()); + code = FdbError.InternalError; + error = e; } - //else: it will be handled below } } + finally + { + FdbNative.FutureDestroy(handle); + } - // most probably the future was cancelled or we are shutting down... - SetCanceled(fromCallback); - } - catch (Exception e) - { // something went wrong - if (e is ThreadAbortException) + if (code == FdbError.Success) + { + TrySetResult(result); + } + else if (code == FdbError.OperationCancelled || code == FdbError.TransactionCancelled) + { + TrySetCanceled(); + } + else { - SetCanceled(fromCallback); - throw; + TrySetException(error ?? Fdb.MapToException(code)); } - SetFaulted(e, fromCallback); } - finally - { -#if DEBUG_FUTURES - sw.Stop(); - Debug.WriteLine("Future<" + typeof(T).Name + "> callback completed in " + sw.Elapsed.TotalMilliseconds.ToString() + " ms"); -#endif - TryCleanup(); + catch (Exception e) + { // we must not blow up the TP or the parent, so make sure to propagate all exceptions to the task + TrySetException(e); } } - protected override void CloseHandles() - { - var handle = m_handle; - if (handle != null) handle.Dispose(); - } - - protected override void CancelHandles() - { - var handle = m_handle; - //REVIEW: there is a possibility of a race condition with Dispose() that could potentially call FutureDestroy(handle) at the same time (not verified) - if (handle != null && !handle.IsClosed && !handle.IsInvalid) FdbNative.FutureCancel(handle); - } - - protected override void ReleaseMemory() - { - var handle = m_handle; - //REVIEW: there is a possibility of a race condition with Dispose() that could potentially call FutureDestroy(handle) at the same time (not verified) - if (handle != null && !handle.IsClosed && !handle.IsInvalid) FdbNative.FutureReleaseMemory(handle); - } - } } diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index c066cb4ec..a3fba2dd4 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -97,7 +97,7 @@ internal static class NativeMethods // Cluster [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl, CharSet = CharSet.Ansi, BestFitMapping = false, ThrowOnUnmappableChar = true)] - public static extern FutureHandle fdb_create_cluster([MarshalAs(UnmanagedType.LPStr)] string clusterFilePath); + public static extern IntPtr fdb_create_cluster([MarshalAs(UnmanagedType.LPStr)] string clusterFilePath); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] public static extern void fdb_cluster_destroy(IntPtr cluster); @@ -106,7 +106,7 @@ internal static class NativeMethods public static extern FdbError fdb_cluster_set_option(ClusterHandle cluster, FdbClusterOption option, byte* value, int valueLength); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl, CharSet = CharSet.Ansi, BestFitMapping = false, ThrowOnUnmappableChar = true)] - public static extern FutureHandle fdb_cluster_create_database(ClusterHandle cluster, [MarshalAs(UnmanagedType.LPStr)] string dbName, int dbNameLength); + public static extern IntPtr fdb_cluster_create_database(ClusterHandle cluster, [MarshalAs(UnmanagedType.LPStr)] string dbName, int dbNameLength); // Database @@ -131,19 +131,19 @@ internal static class NativeMethods public static extern void fdb_transaction_set_read_version(TransactionHandle handle, long version); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_get_read_version(TransactionHandle transaction); + public static extern IntPtr fdb_transaction_get_read_version(TransactionHandle transaction); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_get(TransactionHandle transaction, byte* keyName, int keyNameLength, bool snapshot); + public static extern IntPtr fdb_transaction_get(TransactionHandle transaction, byte* keyName, int keyNameLength, bool snapshot); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_get_addresses_for_key(TransactionHandle transaction, byte* keyName, int keyNameLength); + public static extern IntPtr fdb_transaction_get_addresses_for_key(TransactionHandle transaction, byte* keyName, int keyNameLength); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_get_key(TransactionHandle transaction, byte* keyName, int keyNameLength, bool orEqual, int offset, bool snapshot); + public static extern IntPtr fdb_transaction_get_key(TransactionHandle transaction, byte* keyName, int keyNameLength, bool orEqual, int offset, bool snapshot); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_get_range( + public static extern IntPtr fdb_transaction_get_range( TransactionHandle transaction, byte* beginKeyName, int beginKeyNameLength, bool beginOrEqual, int beginOffset, byte* endKeyName, int endKeyNameLength, bool endOrEqual, int endOffset, @@ -167,16 +167,16 @@ public static extern void fdb_transaction_clear_range( public static extern void fdb_transaction_atomic_op(TransactionHandle transaction, byte* keyName, int keyNameLength, byte* param, int paramLength, FdbMutationType operationType); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_commit(TransactionHandle transaction); + public static extern IntPtr fdb_transaction_commit(TransactionHandle transaction); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] public static extern FdbError fdb_transaction_get_committed_version(TransactionHandle transaction, out long version); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_watch(TransactionHandle transaction, byte* keyName, int keyNameLength); + public static extern IntPtr fdb_transaction_watch(TransactionHandle transaction, byte* keyName, int keyNameLength); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FutureHandle fdb_transaction_on_error(TransactionHandle transaction, FdbError error); + public static extern IntPtr fdb_transaction_on_error(TransactionHandle transaction, FdbError error); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] public static extern void fdb_transaction_reset(TransactionHandle transaction); @@ -193,43 +193,43 @@ public static extern void fdb_transaction_clear_range( public static extern void fdb_future_destroy(IntPtr future); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern void fdb_future_cancel(FutureHandle future); + public static extern void fdb_future_cancel(IntPtr future); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern void fdb_future_release_memory(FutureHandle future); + public static extern void fdb_future_release_memory(IntPtr future); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_block_until_ready(FutureHandle futureHandle); + public static extern FdbError fdb_future_block_until_ready(IntPtr futureHandle); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern bool fdb_future_is_ready(FutureHandle futureHandle); + public static extern bool fdb_future_is_ready(IntPtr futureHandle); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_error(FutureHandle futureHandle); + public static extern FdbError fdb_future_get_error(IntPtr futureHandle); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_set_callback(FutureHandle future, FdbFutureCallback callback, IntPtr callbackParameter); + public static extern FdbError fdb_future_set_callback(IntPtr future, FdbFutureCallback callback, IntPtr callbackParameter); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_version(FutureHandle future, out long version); + public static extern FdbError fdb_future_get_version(IntPtr future, out long version); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_key(FutureHandle future, out byte* key, out int keyLength); + public static extern FdbError fdb_future_get_key(IntPtr future, out byte* key, out int keyLength); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_cluster(FutureHandle future, out ClusterHandle cluster); + public static extern FdbError fdb_future_get_cluster(IntPtr future, out ClusterHandle cluster); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_database(FutureHandle future, out DatabaseHandle database); + public static extern FdbError fdb_future_get_database(IntPtr future, out DatabaseHandle database); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_value(FutureHandle future, out bool present, out byte* value, out int valueLength); + public static extern FdbError fdb_future_get_value(IntPtr future, out bool present, out byte* value, out int valueLength); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_string_array(FutureHandle future, out byte** strings, out int count); + public static extern FdbError fdb_future_get_string_array(IntPtr future, out byte** strings, out int count); [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] - public static extern FdbError fdb_future_get_keyvalue_array(FutureHandle future, out FdbKeyValue* kv, out int count, out bool more); + public static extern FdbError fdb_future_get_keyvalue_array(IntPtr future, out FdbKeyValue* kv, out int count, out bool more); } @@ -346,51 +346,52 @@ public static int GetMaxApiVersion() #region Futures... - public static bool FutureIsReady(FutureHandle futureHandle) + public static bool FutureIsReady(IntPtr futureHandle) { return NativeMethods.fdb_future_is_ready(futureHandle); } - public static void FutureDestroy(IntPtr futureHandle) + public static void FutureDestroy(IntPtr futureHandle, [CallerMemberName] string caller = null) { + Debug.WriteLine("Native.FutureDestroy(0x{0}) from {1}", (object)futureHandle.ToString("X"), caller); if (futureHandle != IntPtr.Zero) { NativeMethods.fdb_future_destroy(futureHandle); } } - public static void FutureCancel(FutureHandle futureHandle) + public static void FutureCancel(IntPtr futureHandle) { NativeMethods.fdb_future_cancel(futureHandle); } - public static void FutureReleaseMemory(FutureHandle futureHandle) + public static void FutureReleaseMemory(IntPtr futureHandle) { NativeMethods.fdb_future_release_memory(futureHandle); } - public static FdbError FutureGetError(FutureHandle future) + public static FdbError FutureGetError(IntPtr future) { return NativeMethods.fdb_future_get_error(future); } - public static FdbError FutureBlockUntilReady(FutureHandle future) + public static FdbError FutureBlockUntilReady(IntPtr future) { #if DEBUG_NATIVE_CALLS - Debug.WriteLine("calling fdb_future_block_until_ready(0x" + future.Handle.ToString("x") + ")..."); + Debug.WriteLine("calling fdb_future_block_until_ready(0x" + future.ToString("x") + ")..."); #endif var err = NativeMethods.fdb_future_block_until_ready(future); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_block_until_ready(0x" + future.Handle.ToString("x") + ") => err=" + err); + Debug.WriteLine("fdb_future_block_until_ready(0x" + future.ToString("x") + ") => err=" + err); #endif return err; } - public static FdbError FutureSetCallback(FutureHandle future, FdbFutureCallback callback, IntPtr callbackParameter) + public static FdbError FutureSetCallback(IntPtr future, FdbFutureCallback callback, IntPtr callbackParameter) { var err = NativeMethods.fdb_future_set_callback(future, callback, callbackParameter); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_set_callback(0x" + future.Handle.ToString("x") + ", 0x" + ptrCallback.ToString("x") + ") => err=" + err); + Debug.WriteLine("fdb_future_set_callback(0x" + future.ToString("x") + ", 0x" + callbackParameter.ToString("x") + ") => err=" + err); #endif return err; } @@ -427,12 +428,12 @@ public static FdbError StopNetwork() #region Clusters... - public static FutureHandle CreateCluster(string path) + public static IntPtr CreateCluster(string path) { var future = NativeMethods.fdb_create_cluster(path); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_create_cluster(" + path + ") => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_create_cluster(" + path + ") => 0x" + future.ToString("x")); #endif return future; @@ -451,11 +452,11 @@ public static FdbError ClusterSetOption(ClusterHandle cluster, FdbClusterOption return NativeMethods.fdb_cluster_set_option(cluster, option, value, valueLength); } - public static FdbError FutureGetCluster(FutureHandle future, out ClusterHandle cluster) + public static FdbError FutureGetCluster(IntPtr future, out ClusterHandle cluster) { var err = NativeMethods.fdb_future_get_cluster(future, out cluster); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_get_cluster(0x" + future.Handle.ToString("x") + ") => err=" + err + ", handle=0x" + cluster.Handle.ToString("x")); + Debug.WriteLine("fdb_future_get_cluster(0x" + future.ToString("x") + ") => err=" + err + ", handle=0x" + cluster.Handle.ToString("x")); #endif //TODO: check if err == Success ? return err; @@ -465,11 +466,11 @@ public static FdbError FutureGetCluster(FutureHandle future, out ClusterHandle c #region Databases... - public static FdbError FutureGetDatabase(FutureHandle future, out DatabaseHandle database) + public static FdbError FutureGetDatabase(IntPtr future, out DatabaseHandle database) { var err = NativeMethods.fdb_future_get_database(future, out database); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_get_database(0x" + future.Handle.ToString("x") + ") => err=" + err + ", handle=0x" + database.Handle.ToString("x")); + Debug.WriteLine("fdb_future_get_database(0x" + future.ToString("x") + ") => err=" + err + ", handle=0x" + database.Handle.ToString("x")); #endif //TODO: check if err == Success ? return err; @@ -488,7 +489,7 @@ public static void DatabaseDestroy(IntPtr handle) } } - public static FutureHandle ClusterCreateDatabase(ClusterHandle cluster, string name) + public static IntPtr ClusterCreateDatabase(ClusterHandle cluster, string name) { var future = NativeMethods.fdb_cluster_create_database(cluster, name, name == null ? 0 : name.Length); Contract.Assert(future != null); @@ -524,17 +525,17 @@ public static FdbError DatabaseCreateTransaction(DatabaseHandle database, out Tr return err; } - public static FutureHandle TransactionCommit(TransactionHandle transaction) + public static IntPtr TransactionCommit(TransactionHandle transaction) { var future = NativeMethods.fdb_transaction_commit(transaction); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_commit(0x" + transaction.Handle.ToString("x") + ") => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_commit(0x" + transaction.Handle.ToString("x") + ") => 0x" + future.ToString("x")); #endif return future; } - public static FutureHandle TransactionWatch(TransactionHandle transaction, Slice key) + public static IntPtr TransactionWatch(TransactionHandle transaction, Slice key) { if (key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty", "key"); @@ -543,18 +544,18 @@ public static FutureHandle TransactionWatch(TransactionHandle transaction, Slice var future = NativeMethods.fdb_transaction_watch(transaction, ptrKey + key.Offset, key.Count); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_watch(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "') => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_watch(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "') => 0x" + future.ToString("x")); #endif return future; } } - public static FutureHandle TransactionOnError(TransactionHandle transaction, FdbError errorCode) + public static IntPtr TransactionOnError(TransactionHandle transaction, FdbError errorCode) { var future = NativeMethods.fdb_transaction_on_error(transaction, errorCode); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_on_error(0x" + transaction.Handle.ToString("x") + ", " + errorCode + ") => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_on_error(0x" + transaction.Handle.ToString("x") + ", " + errorCode + ") => 0x" + future.ToString("x")); #endif return future; } @@ -583,12 +584,12 @@ public static void TransactionSetReadVersion(TransactionHandle transaction, long NativeMethods.fdb_transaction_set_read_version(transaction, version); } - public static FutureHandle TransactionGetReadVersion(TransactionHandle transaction) + public static IntPtr TransactionGetReadVersion(TransactionHandle transaction) { var future = NativeMethods.fdb_transaction_get_read_version(transaction); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_get_read_version(0x" + transaction.Handle.ToString("x") + ") => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_get_read_version(0x" + transaction.Handle.ToString("x") + ") => 0x" + future.ToString("x")); #endif return future; } @@ -601,15 +602,15 @@ public static FdbError TransactionGetCommittedVersion(TransactionHandle transact return NativeMethods.fdb_transaction_get_committed_version(transaction, out version); } - public static FdbError FutureGetVersion(FutureHandle future, out long version) + public static FdbError FutureGetVersion(IntPtr future, out long version) { #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_get_version(0x" + future.Handle.ToString("x") + ")"); + Debug.WriteLine("fdb_future_get_version(0x" + future.ToString("x") + ")"); #endif return NativeMethods.fdb_future_get_version(future, out version); } - public static FutureHandle TransactionGet(TransactionHandle transaction, Slice key, bool snapshot) + public static IntPtr TransactionGet(TransactionHandle transaction, Slice key, bool snapshot) { if (key.IsNull) throw new ArgumentException("Key cannot be null", "key"); @@ -621,13 +622,13 @@ public static FutureHandle TransactionGet(TransactionHandle transaction, Slice k var future = NativeMethods.fdb_transaction_get(transaction, ptrKey + key.Offset, key.Count, snapshot); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_get(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "', snapshot: " + snapshot + ") => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_get(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "', snapshot: " + snapshot + ") => 0x" + future.ToString("x")); #endif return future; } } - public static FutureHandle TransactionGetRange(TransactionHandle transaction, FdbKeySelector begin, FdbKeySelector end, int limit, int targetBytes, FdbStreamingMode mode, int iteration, bool snapshot, bool reverse) + public static IntPtr TransactionGetRange(TransactionHandle transaction, FdbKeySelector begin, FdbKeySelector end, int limit, int targetBytes, FdbStreamingMode mode, int iteration, bool snapshot, bool reverse) { fixed (byte* ptrBegin = begin.Key.Array) fixed (byte* ptrEnd = end.Key.Array) @@ -639,13 +640,13 @@ public static FutureHandle TransactionGetRange(TransactionHandle transaction, Fd limit, targetBytes, mode, iteration, snapshot, reverse); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_get_range(0x" + transaction.Handle.ToString("x") + ", begin: " + begin.PrettyPrint(FdbKey.PrettyPrintMode.Begin) + ", end: " + end.PrettyPrint(FdbKey.PrettyPrintMode.End) + ", " + snapshot + ") => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_get_range(0x" + transaction.Handle.ToString("x") + ", begin: " + begin.PrettyPrint(FdbKey.PrettyPrintMode.Begin) + ", end: " + end.PrettyPrint(FdbKey.PrettyPrintMode.End) + ", " + snapshot + ") => 0x" + future.ToString("x")); #endif return future; } } - public static FutureHandle TransactionGetKey(TransactionHandle transaction, FdbKeySelector selector, bool snapshot) + public static IntPtr TransactionGetKey(TransactionHandle transaction, FdbKeySelector selector, bool snapshot) { if (selector.Key.IsNull) throw new ArgumentException("Key cannot be null", "selector"); @@ -654,13 +655,13 @@ public static FutureHandle TransactionGetKey(TransactionHandle transaction, FdbK var future = NativeMethods.fdb_transaction_get_key(transaction, ptrKey + selector.Key.Offset, selector.Key.Count, selector.OrEqual, selector.Offset, snapshot); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_get_key(0x" + transaction.Handle.ToString("x") + ", " + selector.ToString() + ", " + snapshot + ") => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_get_key(0x" + transaction.Handle.ToString("x") + ", " + selector.ToString() + ", " + snapshot + ") => 0x" + future.ToString("x")); #endif return future; } } - public static FutureHandle TransactionGetAddressesForKey(TransactionHandle transaction, Slice key) + public static IntPtr TransactionGetAddressesForKey(TransactionHandle transaction, Slice key) { if (key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty", "key"); @@ -669,19 +670,19 @@ public static FutureHandle TransactionGetAddressesForKey(TransactionHandle trans var future = NativeMethods.fdb_transaction_get_addresses_for_key(transaction, ptrKey + key.Offset, key.Count); Contract.Assert(future != null); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_transaction_get_addresses_for_key(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "') => 0x" + future.Handle.ToString("x")); + Debug.WriteLine("fdb_transaction_get_addresses_for_key(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "') => 0x" + future.ToString("x")); #endif return future; } } - public static FdbError FutureGetValue(FutureHandle future, out bool valuePresent, out Slice value) + public static FdbError FutureGetValue(IntPtr future, out bool valuePresent, out Slice value) { byte* ptr; int valueLength; var err = NativeMethods.fdb_future_get_value(future, out valuePresent, out ptr, out valueLength); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_get_value(0x" + future.Handle.ToString("x") + ") => err=" + err + ", present=" + valuePresent + ", valueLength=" + valueLength); + Debug.WriteLine("fdb_future_get_value(0x" + future.ToString("x") + ") => err=" + err + ", present=" + valuePresent + ", valueLength=" + valueLength); #endif if (ptr != null && valueLength >= 0) { @@ -696,13 +697,13 @@ public static FdbError FutureGetValue(FutureHandle future, out bool valuePresent return err; } - public static FdbError FutureGetKey(FutureHandle future, out Slice key) + public static FdbError FutureGetKey(IntPtr future, out Slice key) { byte* ptr; int keyLength; var err = NativeMethods.fdb_future_get_key(future, out ptr, out keyLength); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_get_key(0x" + future.Handle.ToString("x") + ") => err=" + err + ", keyLength=" + keyLength); + Debug.WriteLine("fdb_future_get_key(0x" + future.ToString("x") + ") => err=" + err + ", keyLength=" + keyLength); #endif // note: fdb_future_get_key is allowed to return NULL for the empty key (not to be confused with a key that has an empty value) @@ -719,7 +720,7 @@ public static FdbError FutureGetKey(FutureHandle future, out Slice key) return err; } - public static FdbError FutureGetKeyValueArray(FutureHandle future, out KeyValuePair[] result, out bool more) + public static FdbError FutureGetKeyValueArray(IntPtr future, out KeyValuePair[] result, out bool more) { result = null; @@ -728,7 +729,7 @@ public static FdbError FutureGetKeyValueArray(FutureHandle future, out KeyValueP var err = NativeMethods.fdb_future_get_keyvalue_array(future, out kvp, out count, out more); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_get_keyvalue_array(0x" + future.Handle.ToString("x") + ") => err=" + err + ", count=" + count + ", more=" + more); + Debug.WriteLine("fdb_future_get_keyvalue_array(0x" + future.ToString("x") + ") => err=" + err + ", count=" + count + ", more=" + more); #endif if (Fdb.Success(err)) @@ -789,7 +790,7 @@ public static FdbError FutureGetKeyValueArray(FutureHandle future, out KeyValueP return err; } - public static FdbError FutureGetStringArray(FutureHandle future, out string[] result) + public static FdbError FutureGetStringArray(IntPtr future, out string[] result) { result = null; @@ -798,7 +799,7 @@ public static FdbError FutureGetStringArray(FutureHandle future, out string[] re var err = NativeMethods.fdb_future_get_string_array(future, out strings, out count); #if DEBUG_NATIVE_CALLS - Debug.WriteLine("fdb_future_get_string_array(0x" + future.Handle.ToString("x") + ") => err=" + err + ", count=" + count); + Debug.WriteLine("fdb_future_get_string_array(0x" + future.ToString("x") + ") => err=" + err + ", count=" + count); #endif if (Fdb.Success(err)) diff --git a/FoundationDB.Client/Native/FdbNativeCluster.cs b/FoundationDB.Client/Native/FdbNativeCluster.cs index 1a8831418..7c5d8e42a 100644 --- a/FoundationDB.Client/Native/FdbNativeCluster.cs +++ b/FoundationDB.Client/Native/FdbNativeCluster.cs @@ -37,37 +37,59 @@ namespace FoundationDB.Client.Native using System.Threading.Tasks; /// Wraps a native FDBCluster* handle - internal sealed class FdbNativeCluster : IFdbClusterHandler + internal sealed class FdbNativeCluster : FdbFutureContext, IFdbClusterHandler { - private readonly ClusterHandle m_handle; + //private readonly ClusterHandle m_handle; public FdbNativeCluster(ClusterHandle handle) + : base(handle) { - Contract.Requires(handle != null); - m_handle = handle; } - public static Task CreateClusterAsync(string clusterFile, CancellationToken cancellationToken) + private static readonly GlobalNativeContext GlobalContext = new GlobalNativeContext(); + + private sealed class GlobalNativeContext : FdbFutureContext { - var future = FdbNative.CreateCluster(clusterFile); - return FdbFuture.CreateTaskFromHandle(future, - (h) => + + public Task CreateClusterAsync(string clusterFile, CancellationToken ct) + { + IntPtr handle = IntPtr.Zero; + bool dead = true; + try { - ClusterHandle cluster; - var err = FdbNative.FutureGetCluster(h, out cluster); - if (err != FdbError.Success) - { - cluster.Dispose(); - throw Fdb.MapToException(err); - } - var handler = new FdbNativeCluster(cluster); - return (IFdbClusterHandler) handler; - }, - cancellationToken - ); + return RegisterFuture( + FdbNative.CreateCluster(clusterFile), + ref dead, + (h, state) => + { + ClusterHandle cluster; + var err = FdbNative.FutureGetCluster(h, out cluster); + if (err != FdbError.Success) + { + cluster.Dispose(); + throw Fdb.MapToException(err); + } + var handler = new FdbNativeCluster(cluster); + return (IFdbClusterHandler) handler; + }, + null, + ct, + "CreateClusterAsync" + ); + } + finally + { + if (handle != IntPtr.Zero && dead) FdbNative.FutureDestroy(handle); + } + } + } - internal ClusterHandle Handle { get { return m_handle; } } + + public static Task CreateClusterAsync(string clusterFile, CancellationToken cancellationToken) + { + return GlobalContext.CreateClusterAsync(clusterFile, cancellationToken); + } public bool IsInvalid { get { return m_handle.IsInvalid; } } @@ -97,10 +119,10 @@ public Task OpenDatabaseAsync(string databaseName, Cancella { if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); - var future = FdbNative.ClusterCreateDatabase(m_handle, databaseName); - return FdbFuture.CreateTaskFromHandle( - future, - (h) => + return StartNewFuture( + (handle, state) => FdbNative.ClusterCreateDatabase(handle, state), + databaseName, + (h, state) => { DatabaseHandle database; var err = FdbNative.FutureGetDatabase(h, out database); @@ -109,18 +131,14 @@ public Task OpenDatabaseAsync(string databaseName, Cancella database.Dispose(); throw Fdb.MapToException(err); } - var handler = new FdbNativeDatabase(database); + var handler = new FdbNativeDatabase(database, (string)state); return (IFdbDatabaseHandler) handler; }, + databaseName, cancellationToken ); } - public void Dispose() - { - if (m_handle != null) m_handle.Dispose(); - } - } diff --git a/FoundationDB.Client/Native/FdbNativeDatabase.cs b/FoundationDB.Client/Native/FdbNativeDatabase.cs index 2ba7e47c5..1cc8baf52 100644 --- a/FoundationDB.Client/Native/FdbNativeDatabase.cs +++ b/FoundationDB.Client/Native/FdbNativeDatabase.cs @@ -37,43 +37,20 @@ namespace FoundationDB.Client.Native /// Wraps a native FDBDatabase* handle [DebuggerDisplay("Handle={m_handle}, Closed={m_handle.IsClosed}")] - internal sealed class FdbNativeDatabase : IFdbDatabaseHandler + internal sealed class FdbNativeDatabase : FdbFutureContext, IFdbDatabaseHandler { - /// Handle that wraps the native FDB_DATABASE* - private readonly DatabaseHandle m_handle; - -#if CAPTURE_STACKTRACES - private readonly StackTrace m_stackTrace; -#endif - - public FdbNativeDatabase(DatabaseHandle handle) - { - if (handle == null) throw new ArgumentNullException("handle"); - - m_handle = handle; -#if CAPTURE_STACKTRACES - m_stackTrace = new StackTrace(); -#endif - } - - //REVIEW: do we really need a destructor ? The handle is a SafeHandle, and will take care of itself... - ~FdbNativeDatabase() + public FdbNativeDatabase(DatabaseHandle handle, string name) + : base(handle) { -#if CAPTURE_STACKTRACES - Trace.WriteLine("A database handle (" + m_handle + ") was leaked by " + m_stackTrace); -#endif -#if DEBUG - // If you break here, that means that a native database handler was leaked by a FdbDatabase instance (or that the database instance was leaked) - if (Debugger.IsAttached) Debugger.Break(); -#endif - Dispose(false); + this.Name = name; } - public bool IsInvalid { get { return m_handle.IsInvalid; } } public bool IsClosed { get { return m_handle.IsClosed; } } + public string Name { get; private set; } + public void SetOption(FdbDatabaseOption option, Slice data) { Fdb.EnsureNotOnNetworkThread(); @@ -113,19 +90,6 @@ public IFdbTransactionHandler CreateTransaction(FdbOperationContext context) } } - public void Dispose() - { - Dispose(true); - GC.SuppressFinalize(this); - } - - private void Dispose(bool disposing) - { - if (disposing) - { - if (m_handle != null) m_handle.Dispose(); - } - } } } diff --git a/FoundationDB.Client/Native/FdbNativeTransaction.cs b/FoundationDB.Client/Native/FdbNativeTransaction.cs index 63317a1b1..43f5ad3a7 100644 --- a/FoundationDB.Client/Native/FdbNativeTransaction.cs +++ b/FoundationDB.Client/Native/FdbNativeTransaction.cs @@ -35,6 +35,7 @@ namespace FoundationDB.Client.Native { using FoundationDB.Client.Core; using FoundationDB.Client.Utils; + using FoundationDB.Layers.Tuples; using JetBrains.Annotations; using System; using System.Collections.Generic; @@ -44,50 +45,25 @@ namespace FoundationDB.Client.Native /// Wraps a native FDB_TRANSACTION handle [DebuggerDisplay("Handle={m_handle}, Size={m_payloadBytes}, Closed={m_handle.IsClosed}")] - internal class FdbNativeTransaction : IFdbTransactionHandler + internal class FdbNativeTransaction : FdbFutureContext, IFdbTransactionHandler { + private readonly FdbNativeDatabase m_database; - /// FDB_TRANSACTION* handle - private readonly TransactionHandle m_handle; /// Estimated current size of the transaction private int m_payloadBytes; -#if CAPTURE_STACKTRACES - private StackTrace m_stackTrace; -#endif - - public FdbNativeTransaction(FdbNativeDatabase db, TransactionHandle handle) + public FdbNativeTransaction([NotNull] FdbNativeDatabase db, [NotNull] TransactionHandle handle) + : base(handle) { if (db == null) throw new ArgumentNullException("db"); - if (handle == null) throw new ArgumentNullException("handle"); m_database = db; - m_handle = handle; -#if CAPTURE_STACKTRACES - m_stackTrace = new StackTrace(); -#endif - } - - //REVIEW: do we really need a destructor ? The handle is a SafeHandle, and will take care of itself... - ~FdbNativeTransaction() - { -#if CAPTURE_STACKTRACES - Trace.WriteLine("A transaction handle (" + m_handle + ", " + m_payloadBytes + " bytes written) was leaked by " + m_stackTrace); -#endif -#if DEBUG - // If you break here, that means that a native transaction handler was leaked by a FdbTransaction instance (or that the transaction instance was leaked) - if (Debugger.IsAttached) Debugger.Break(); -#endif - Dispose(false); } #region Properties... public bool IsClosed { get { return m_handle.IsClosed; } } - /// Native FDB_TRANSACTION* handle - public TransactionHandle Handle { get { return m_handle; } } - /// Database handler that owns this transaction public FdbNativeDatabase Database { get { return m_database; } } @@ -133,18 +109,20 @@ public void SetOption(FdbTransactionOption option, Slice data) public Task GetReadVersionAsync(CancellationToken cancellationToken) { - var future = FdbNative.TransactionGetReadVersion(m_handle); - return FdbFuture.CreateTaskFromHandle(future, - (h) => + return StartNewFuture( + (handle, state) => FdbNative.TransactionGetReadVersion(handle), + default(object), + (future, state) => { long version; - var err = FdbNative.FutureGetVersion(h, out version); + var err = FdbNative.FutureGetVersion(future, out version); #if DEBUG_TRANSACTIONS Debug.WriteLine("FdbTransaction[" + m_id + "].GetReadVersion() => err=" + err + ", version=" + version); #endif Fdb.DieOnError(err); return version; }, + default(object), cancellationToken ); } @@ -154,10 +132,8 @@ public void SetReadVersion(long version) FdbNative.TransactionSetReadVersion(m_handle, version); } - private static bool TryGetValueResult(FutureHandle h, out Slice result) + private static bool TryGetValueResult(IntPtr h, out Slice result) { - Contract.Requires(h != null); - bool present; var err = FdbNative.FutureGetValue(h, out present, out result); #if DEBUG_TRANSACTIONS @@ -167,22 +143,21 @@ private static bool TryGetValueResult(FutureHandle h, out Slice result) return present; } - private static Slice GetValueResultBytes(FutureHandle h) + private static Slice GetValueResultBytes(IntPtr h) { - Contract.Requires(h != null); - Slice result; - if (!TryGetValueResult(h, out result)) - { - return Slice.Nil; - } - return result; + return !TryGetValueResult(h, out result) ? Slice.Nil : result; } public Task GetAsync(Slice key, bool snapshot, CancellationToken cancellationToken) { - var future = FdbNative.TransactionGet(m_handle, key, snapshot); - return FdbFuture.CreateTaskFromHandle(future, (h) => GetValueResultBytes(h), cancellationToken); + return StartNewFuture( + (handle, state) => FdbNative.TransactionGet(handle, state.Item1, state.Item2), + FdbTuple.Create(key, snapshot), + (future, state) => GetValueResultBytes(future), + null, + cancellationToken + ); } public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationToken cancellationToken) @@ -191,24 +166,24 @@ public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationTok if (keys.Length == 0) return Task.FromResult(Slice.EmptySliceArray); - var futures = new FutureHandle[keys.Length]; - try - { - for (int i = 0; i < keys.Length; i++) - { - futures[i] = FdbNative.TransactionGet(m_handle, keys[i], snapshot); - } - } - catch - { - for (int i = 0; i < keys.Length; i++) + return StartNewFutures( + keys.Length, + (handle, state, futures) => { - if (futures[i] == null) break; - futures[i].Dispose(); - } - throw; - } - return FdbFuture.CreateTaskFromHandleArray(futures, (h) => GetValueResultBytes(h), cancellationToken); + var _keys = state.Item1; + var _snapshot = state.Item2; + for (int i = 0; i < _keys.Length; i++) + { + var h = FdbNative.TransactionGet(handle, _keys[i], _snapshot); + if (h == IntPtr.Zero) throw new FdbException(FdbError.OperationFailed); + futures[i] = h; + } + }, + FdbTuple.Create(keys, snapshot), + (future, state) => GetValueResultBytes(future), + default(object), //TODO: buffer for the slices + cancellationToken + ); } /// Extract a chunk of result from a completed Future @@ -216,7 +191,7 @@ public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationTok /// Receives true if there are more result, or false if all results have been transmited /// Array of key/value pairs, or an exception [NotNull] - private static KeyValuePair[] GetKeyValueArrayResult(FutureHandle h, out bool more) + private static KeyValuePair[] GetKeyValueArrayResult(IntPtr h, out bool more) { KeyValuePair[] result; var err = FdbNative.FutureGetKeyValueArray(h, out result, out more); @@ -233,26 +208,26 @@ public Task GetRangeAsync(FdbKeySelector begin, FdbKeySelector en Contract.Requires(options != null); bool reversed = options.Reverse ?? false; - var future = FdbNative.TransactionGetRange(m_handle, begin, end, options.Limit ?? 0, options.TargetBytes ?? 0, options.Mode ?? FdbStreamingMode.Iterator, iteration, snapshot, reversed); - return FdbFuture.CreateTaskFromHandle( - future, - (h) => + + return StartNewFuture( + (handle, _) => FdbNative.TransactionGetRange(handle, begin, end, options.Limit ?? 0, options.TargetBytes ?? 0, options.Mode ?? FdbStreamingMode.Iterator, iteration, snapshot, reversed), + default(object), //TODO: pass options & co? + (future, state) => { // TODO: quietly return if disposed bool hasMore; - var chunk = GetKeyValueArrayResult(h, out hasMore); + var chunk = GetKeyValueArrayResult(future, out hasMore); return new FdbRangeChunk(hasMore, chunk, iteration, reversed); }, + default(object), //TODO: pass options & co? cancellationToken ); } - private static Slice GetKeyResult(FutureHandle h) + private static Slice GetKeyResult(IntPtr h) { - Contract.Requires(h != null); - Slice result; var err = FdbNative.FutureGetKey(h, out result); #if DEBUG_TRANSACTIONS @@ -264,10 +239,11 @@ private static Slice GetKeyResult(FutureHandle h) public Task GetKeyAsync(FdbKeySelector selector, bool snapshot, CancellationToken cancellationToken) { - var future = FdbNative.TransactionGetKey(m_handle, selector, snapshot); - return FdbFuture.CreateTaskFromHandle( - future, - (h) => GetKeyResult(h), + return StartNewFuture( + (handle, state) => FdbNative.TransactionGetKey(handle, state.Item1, state.Item2), + FdbTuple.Create(selector, snapshot), + (future, state) => GetKeyResult(future), + default(object), cancellationToken ); } @@ -276,25 +252,26 @@ public Task GetKeysAsync(FdbKeySelector[] selectors, bool snapshot, Can { Contract.Requires(selectors != null); - var futures = new FutureHandle[selectors.Length]; - try - { - for (int i = 0; i < selectors.Length; i++) - { - futures[i] = FdbNative.TransactionGetKey(m_handle, selectors[i], snapshot); - } - } - catch - { - for (int i = 0; i < selectors.Length; i++) - { - if (futures[i] == null) break; - futures[i].Dispose(); - } - throw; - } - return FdbFuture.CreateTaskFromHandleArray(futures, (h) => GetKeyResult(h), cancellationToken); + if (selectors.Length == 0) return Task.FromResult(Slice.EmptySliceArray); + return StartNewFutures( + selectors.Length, + (handle, state, futures) => + { + var _selectors = state.Item1; + var _snapshot = state.Item2; + for (int i = 0; i < _selectors.Length; i++) + { + var h = FdbNative.TransactionGetKey(handle, _selectors[i], _snapshot); + if (h == IntPtr.Zero) throw new FdbException(FdbError.OperationFailed); + futures[i] = h; + } + }, + FdbTuple.Create(selectors, snapshot), + (future, state) => GetKeyResult(future), + default(object), //TODO: buffer for the slices + cancellationToken + ); } #endregion @@ -340,10 +317,8 @@ public void AddConflictRange(Slice beginKeyInclusive, Slice endKeyExclusive, Fdb } [NotNull] - private static string[] GetStringArrayResult(FutureHandle h) + private static string[] GetStringArrayResult(IntPtr h) { - Contract.Requires(h != null); - string[] result; var err = FdbNative.FutureGetStringArray(h, out result); #if DEBUG_TRANSACTIONS @@ -356,10 +331,11 @@ private static string[] GetStringArrayResult(FutureHandle h) public Task GetAddressesForKeyAsync(Slice key, CancellationToken cancellationToken) { - var future = FdbNative.TransactionGetAddressesForKey(m_handle, key); - return FdbFuture.CreateTaskFromHandle( - future, - (h) => GetStringArrayResult(h), + return StartNewFuture( + (handle, state) => FdbNative.TransactionGetAddressesForKey(handle, state), + key, + (future, state) => GetStringArrayResult(future), + default(object), cancellationToken ); } @@ -370,12 +346,13 @@ public Task GetAddressesForKeyAsync(Slice key, CancellationToken cance public FdbWatch Watch(Slice key, CancellationToken cancellationToken) { - var future = FdbNative.TransactionWatch(m_handle, key); - return new FdbWatch( - FdbFuture.FromHandle(future, (h) => key, cancellationToken), - key, - Slice.Nil - ); + throw new NotImplementedException("FIXME: Future refactoring in progress! I owe you a beer (*) if I ever forget to remove this before committing! (*: if you come get it in person!)"); + //var future = FdbNative.TransactionWatch(m_handle, key); + //return new FdbWatch( + // FdbFuture.FromHandle(future, (h) => key, cancellationToken), + // key, + // Slice.Nil + //); } #endregion @@ -402,14 +379,28 @@ public long GetCommittedVersion() /// As with other client/server databases, in some failure scenarios a client may be unable to determine whether a transaction succeeded. In these cases, CommitAsync() will throw CommitUnknownResult error. The OnErrorAsync() function treats this error as retryable, so retry loops that don’t check for CommitUnknownResult could execute the transaction twice. In these cases, you must consider the idempotence of the transaction. public Task CommitAsync(CancellationToken cancellationToken) { - var future = FdbNative.TransactionCommit(m_handle); - return FdbFuture.CreateTaskFromHandle(future, (h) => null, cancellationToken); + return StartNewFuture( + (handle, state) => FdbNative.TransactionCommit(handle), + default(object), + (future, state) => state, + default(object), //TODO:? + cancellationToken + ); } public Task OnErrorAsync(FdbError code, CancellationToken cancellationToken) { - var future = FdbNative.TransactionOnError(m_handle, code); - return FdbFuture.CreateTaskFromHandle(future, (h) => { ResetInternal(); return null; }, cancellationToken); + return StartNewFuture( + (handle, state) => FdbNative.TransactionOnError(handle, state), + code, + (h, state) => + { + ((FdbNativeTransaction)state).ResetInternal(); + return default(object); + }, + this, + cancellationToken + ); } public void Reset() @@ -430,25 +421,6 @@ private void ResetInternal() #endregion - #region IDisposable... - - public void Dispose() - { - Dispose(true); - GC.SuppressFinalize(this); - } - - private void Dispose(bool disposing) - { - if (disposing) - { - // Dispose of the handle - if (!m_handle.IsClosed) m_handle.Dispose(); - } - } - - #endregion - } } diff --git a/FoundationDB.Client/Native/Handles/FutureHandle.cs b/FoundationDB.Client/Native/Handles/FutureHandle.cs index bc90a5a06..22b2c75aa 100644 --- a/FoundationDB.Client/Native/Handles/FutureHandle.cs +++ b/FoundationDB.Client/Native/Handles/FutureHandle.cs @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +#if REFACTORED + namespace FoundationDB.Client.Native { using FoundationDB.Client.Utils; @@ -62,3 +64,5 @@ public override string ToString() } } + +#endif \ No newline at end of file diff --git a/FoundationDB.Client/Native/IFdbFuture.cs b/FoundationDB.Client/Native/IFdbFuture.cs new file mode 100644 index 000000000..317037938 --- /dev/null +++ b/FoundationDB.Client/Native/IFdbFuture.cs @@ -0,0 +1,53 @@ +#region BSD Licence +/* Copyright (c) 2013-2015, Doxense SAS +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of Doxense nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ +#endregion + +// enable this to help debug Futures +#undef DEBUG_FUTURES + +namespace FoundationDB.Client.Native +{ + using System; + + internal interface IFdbFuture + { + /// Unique identifier of this future + IntPtr Cookie { get; } + + /// Label of the future (usually the name of the operation) + string Label { get; } + + /// Test if this was the last pending handle for this future, or not + /// Handle that completed + /// True if this was the last handle and OnFired() can be called, or False if more handles need to fire first. + bool Visit(IntPtr handle); + + /// Called when all handles tracked by this future have fired + void OnFired(); + } + +} From 6c3c00afec31c01891df0d35e970f410e6aed52c Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 28 Jan 2015 14:02:08 +0100 Subject: [PATCH 02/13] A bit of refactoring and cleanup --- FoundationDB.Client/Native/FdbFuture.cs | 91 +++------ FoundationDB.Client/Native/FdbFutureArray.cs | 180 +++++++++-------- .../Native/FdbFutureContext.cs | 186 +++++++++++++++--- FoundationDB.Client/Native/FdbFutureSingle.cs | 100 +++++----- FoundationDB.Client/Native/IFdbFuture.cs | 7 +- 5 files changed, 334 insertions(+), 230 deletions(-) diff --git a/FoundationDB.Client/Native/FdbFuture.cs b/FoundationDB.Client/Native/FdbFuture.cs index 0827e7adc..4a0387d7e 100644 --- a/FoundationDB.Client/Native/FdbFuture.cs +++ b/FoundationDB.Client/Native/FdbFuture.cs @@ -29,11 +29,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY // enable this to help debug Futures #undef DEBUG_FUTURES +using System.Diagnostics.Contracts; + namespace FoundationDB.Client.Native { using System; using System.Diagnostics; using System.Runtime.CompilerServices; + using System.Threading; using System.Threading.Tasks; /// Base class for all FDBFuture wrappers @@ -44,11 +47,12 @@ internal abstract class FdbFuture : TaskCompletionSource, IFdbFuture #region Private Members... - ///// Optionnal registration on the parent Cancellation Token - ///// Is only valid if FLAG_HAS_CTR is set - //protected CancellationTokenRegistration m_ctr; + /// Optionnal registration on the parent Cancellation Token + /// Is only valid if FLAG_HAS_CTR is set + internal CancellationTokenRegistration m_ctr; - protected FdbFuture(IntPtr cookie, string label) + protected FdbFuture(IntPtr cookie, string label, object state) + : base(state) { this.Cookie = cookie; this.Label = label; @@ -62,45 +66,11 @@ protected FdbFuture(IntPtr cookie, string label) #region Cancellation... -#if REFACTORED - - protected void RegisterForCancellation(CancellationToken cancellationToken) - { - //note: if the token is already cancelled, the callback handler will run inline and any exception would bubble up here - //=> this is not a problem because the ctor already has a try/catch that will clean up everything - m_ctr = cancellationToken.Register( - (_state) => { CancellationHandler(_state); }, - this, - false - ); - } - - protected void UnregisterCancellationRegistration() - { - // unsubscribe from the parent cancellation token if there was one - m_ctr.Dispose(); - m_ctr = default(CancellationTokenRegistration); - } - - private static void CancellationHandler(object state) - { - var future = state as FdbFuture; - if (future != null) - { -#if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + ">.Cancel(0x" + future.m_handle.Handle.ToString("x") + ") was called on thread #" + Thread.CurrentThread.ManagedThreadId.ToString()); -#endif - future.Cancel(); - } - } - -#endif - #endregion public abstract bool Visit(IntPtr handle); - public abstract void OnFired(); + public abstract void OnReady(); /// Return true if the future has completed (successfully or not) public bool IsReady @@ -117,30 +87,33 @@ public TaskAwaiter GetAwaiter() /// Try to abort the task (if it is still running) public void Cancel() { - throw new NotImplementedException("FIXME: Future Cancellation!"); -#if REFACTORED - if (HasAnyFlags(FdbFuture.Flags.DISPOSED | FdbFuture.Flags.COMPLETED | FdbFuture.Flags.CANCELLED)) + if (this.Task.IsCanceled) return; + + OnCancel(); + } + + protected abstract void OnCancel(); + + protected void PublishResult(T result) + { + TrySetResult(result); + } + + protected void PublishError(Exception error, FdbError code) + { + if (error != null) { - return; + TrySetException(error); } - - if (TrySetFlag(FdbFuture.Flags.CANCELLED)) + else if (FdbFutureContext.ClassifyErrorSeverity(code) == FdbFutureContext.CATEGORY_CANCELLED) + { + TrySetCanceled(); + } + else { - bool fromCallback = Fdb.IsNetworkThread; - try - { - if (!this.Task.IsCompleted) - { - CancelHandles(); - SetCanceled(fromCallback); - } - } - finally - { - TryCleanup(); - } + Contract.Assert(code != FdbError.Success); + TrySetException(Fdb.MapToException(code)); } -#endif } } diff --git a/FoundationDB.Client/Native/FdbFutureArray.cs b/FoundationDB.Client/Native/FdbFutureArray.cs index bffcf5c0a..72b0a0c45 100644 --- a/FoundationDB.Client/Native/FdbFutureArray.cs +++ b/FoundationDB.Client/Native/FdbFutureArray.cs @@ -37,26 +37,36 @@ namespace FoundationDB.Client.Native /// Type of result internal sealed class FdbFutureArray : FdbFuture { + // This future encapsulate multiple FDBFuture* handles and use ref-counting to detect when all the handles have fired + // The ref-counting is handled by the network thread, and invokation of future.OnReady() is deferred to the ThreadPool once the counter reaches zero + // The result array is computed once all FDBFuture are ready, from the ThreadPool. + // If at least one of the FDBFuture fails, the Task fails, using the most "serious" error found (ie: Non-Retryable > Cancelled > Retryable) #region Private Members... + /// Encapsulated handles + // May contains IntPtr.Zero handles if there was a problem when setting up the callbacks. + // Atomically set to null by the first thread that needs to destroy all the handles + [CanBeNull] private IntPtr[] m_handles; + /// Number of handles that haven't fired yet private int m_pending; + /// Lambda used to extract the result of one handle + // the first argument is the FDBFuture handle that must be ready and not failed + // the second argument is a state that is passed by the caller. + [NotNull] private readonly Func m_resultSelector; - private readonly object m_state; - #endregion internal FdbFutureArray([NotNull] IntPtr[] handles, [NotNull] Func selector, object state, IntPtr cookie, string label) - : base(cookie, label) + : base(cookie, label, state) { m_handles = handles; m_pending = handles.Length; m_resultSelector = selector; - m_state = state; } public override bool Visit(IntPtr handle) @@ -64,128 +74,116 @@ public override bool Visit(IntPtr handle) return 0 == Interlocked.Decrement(ref m_pending); } - private const int CATEGORY_SUCCESS = 0; - private const int CATEGORY_RETRYABLE = 1; - private const int CATEGORY_CANCELLED = 2; - private const int CATEGORY_FAILURE = 3; - - private static int ClassifyErrorSeverity(FdbError error) + public override void OnReady() { - switch (error) - { - case FdbError.Success: - return CATEGORY_SUCCESS; - - case FdbError.PastVersion: - case FdbError.FutureVersion: - case FdbError.TimedOut: - case FdbError.TooManyWatches: - return CATEGORY_RETRYABLE; - - case FdbError.OperationCancelled: - case FdbError.TransactionCancelled: - return CATEGORY_CANCELLED; - - default: - return CATEGORY_FAILURE; - } - } - - public override void OnFired() - { - var handles = Interlocked.Exchange(ref m_handles, null); - if (handles == null) return; // already disposed? - - Debug.WriteLine("Future{0}<{1}[]>.OnFired({2})", this.Label, typeof (T).Name, handles.Length); - //README: // - This callback will fire either from the ThreadPool (async ops) or inline form the ctor of the future (non-async ops, or ops that where served from some cache). // - The method *MUST* dispose the future handle before returning, and *SHOULD* do so before signaling the task. // => This is because continuations may run inline, and start new futures from there, while we still have our original future handle opened. + IntPtr[] handles = null; try { + // make sure that nobody can destroy our handles while we are using them. + handles = Interlocked.Exchange(ref m_handles, null); + if (handles == null) return; // already disposed? + + Debug.WriteLine("FutureArray.{0}<{1}[]>.OnReady([{2}])", this.Label, typeof(T).Name, handles.Length); + T[] results = new T[handles.Length]; FdbError code = FdbError.Success; int severity = 0; Exception error = null; - try - { - if (this.Task.IsCompleted) - { // task has already been handled by someone else - return; - } - for (int i = 0; i < results.Length; i++) + if (this.Task.IsCompleted) + { // task has already been handled by someone else + return; + } + + var state = this.Task.AsyncState; + for (int i = 0; i < results.Length; i++) + { + var handle = handles[i]; + var err = FdbNative.FutureGetError(handle); + if (err == FdbError.Success) { - var handle = handles[i]; - var err = FdbNative.FutureGetError(handle); - if (err == FdbError.Success) + if (code != FdbError.Success) + { // there's been at least one error before, so there is no point in computing the result, it would be discarded anyway + continue; + } + + try { - if (code != FdbError.Success) - { // there's been at least one error before, so there is no point in computing the result, it would be discarded anyway - continue; - } - - try - { - results[i] = m_resultSelector(handle, m_state); - } - catch (AccessViolationException e) - { // trouble in paradise! - - Debug.WriteLine("EPIC FAIL: " + e.ToString()); - - // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) - if (Debugger.IsAttached) Debugger.Break(); - - Environment.FailFast("FIXME: FDB done goofed!", e); - } - catch (Exception e) - { - Debug.WriteLine("FAIL: " + e.ToString()); - code = FdbError.InternalError; - error = e; - break; - } + results[i] = m_resultSelector(handle, state); } - else if (code != err) + catch (AccessViolationException e) + { // trouble in paradise! + + Debug.WriteLine("EPIC FAIL: " + e.ToString()); + + // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) + if (Debugger.IsAttached) Debugger.Break(); + + Environment.FailFast("FIXME: FDB done goofed!", e); + } + catch (Exception e) { - int cur = ClassifyErrorSeverity(err); - if (cur > severity) - { // error is more serious than before - severity = cur; - code = err; - } + Debug.WriteLine("FAIL: " + e.ToString()); + code = FdbError.InternalError; + error = e; + break; } } - } - finally - { - foreach (var handle in handles) + else if (code != err) { - if (handle != IntPtr.Zero) FdbNative.FutureDestroy(handle); + int cur = FdbFutureContext.ClassifyErrorSeverity(err); + if (cur > severity) + { // error is more serious than before + severity = cur; + code = err; + } } } + // since continuations may fire inline, make sure to release all the memory used by this handle first + FdbFutureContext.DestroyHandles(ref handles); + if (code == FdbError.Success) { - TrySetResult(results); - } - else if (code == FdbError.OperationCancelled || code == FdbError.TransactionCancelled) - { - TrySetCanceled(); + PublishResult(results); } else { - TrySetException(error ?? Fdb.MapToException(code)); + PublishError(error, code); } } catch (Exception e) { // we must not blow up the TP or the parent, so make sure to propagate all exceptions to the task TrySetException(e); } + finally + { + if (handles != null) FdbFutureContext.DestroyHandles(ref handles); + GC.KeepAlive(this); + } } + + protected override void OnCancel() + { + var handles = Volatile.Read(ref m_handles); + //TODO: we probably need locking to prevent concurrent destroy and cancel calls + if (handles != null) + { + foreach (var handle in handles) + { + if (handle != IntPtr.Zero) + { + FdbNative.FutureCancel(handle); + } + } + } + } + } } \ No newline at end of file diff --git a/FoundationDB.Client/Native/FdbFutureContext.cs b/FoundationDB.Client/Native/FdbFutureContext.cs index 2dba49c7a..ea7157c28 100644 --- a/FoundationDB.Client/Native/FdbFutureContext.cs +++ b/FoundationDB.Client/Native/FdbFutureContext.cs @@ -29,13 +29,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY // enable this to capture the stacktrace of the ctor, when troubleshooting leaked transaction handles #undef CAPTURE_STACKTRACES -using System.IO.IsolatedStorage; +using FoundationDB.Async; namespace FoundationDB.Client.Native { - using FoundationDB.Client.Core; using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; using JetBrains.Annotations; using System; using System.Collections.Generic; @@ -89,7 +87,7 @@ private static void FutureCallbackHandler(IntPtr handle, IntPtr cookie) Debug.WriteLine("FutureCallbackHandler(0x{0}, {1:X8} | {2:X8}) called", handle.ToString("X"), cookie.ToInt64() >> 32, cookie.ToInt64() & uint.MaxValue); - bool deferred = false; + bool keepAlive = false; try { @@ -104,12 +102,12 @@ private static void FutureCallbackHandler(IntPtr handle, IntPtr cookie) if (context != null) { Contract.Assert(context.m_contextId == contextId); - deferred = context.OnFired(handle, cookie); + keepAlive = context.OnFutureReady(handle, cookie); } } finally { - if (!deferred) FdbNative.FutureDestroy(handle); + if (!keepAlive) DestroyHandle(ref handle); } } @@ -189,8 +187,7 @@ protected virtual void Dispose(bool disposing) /// A callback has fire for a future handled by this context /// /// - /// If this flag is set to true, then the caller will NOT destroy the future. - private bool OnFired(IntPtr handle, IntPtr cookie) + private bool OnFutureReady(IntPtr handle, IntPtr cookie) { IFdbFuture future; lock (m_futures) @@ -205,7 +202,7 @@ private bool OnFired(IntPtr handle, IntPtr cookie) ThreadPool.UnsafeQueueUserWorkItem( (state) => { - ((IFdbFuture)state).OnFired(); + ((IFdbFuture)state).OnReady(); //TODO: if it fails, maybe we should remove it from m_futures? }, future @@ -237,23 +234,40 @@ protected Task RegisterFuture( string label ) { + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + FdbFutureSingle future = null; IntPtr cookie = IntPtr.Zero; try { - uint futureId = (uint)Interlocked.Increment(ref m_localCookieCounter); + uint futureId = (uint) Interlocked.Increment(ref m_localCookieCounter); cookie = MakeCallbackCookie(m_contextId, futureId); future = new FdbFutureSingle(handle, selector, state, cookie, label); if (FdbNative.FutureIsReady(handle)) { // the result is already computed - Debug.WriteLine("Future.{0} 0x{1} already completed!", label, handle.ToString("X")); + Debug.WriteLine("FutureSingle.{0} 0x{1} already completed!", label, handle.ToString("X")); + cookie = IntPtr.Zero; mustDispose = false; - future.OnFired(); + future.OnReady(); return future.Task; } + if (ct.CanBeCanceled) + { + if (ct.IsCancellationRequested) + { + future.TrySetCanceled(); + cookie = IntPtr.Zero; + return future.Task; + } + + // note that the cancellation handler can fire inline, but it will only mark the future as cancelled + // this means that we will still wait for the future callback to fire and set the task state in there. + future.m_ctr = RegisterForCancellation(future, ct); + } + lock (m_futures) { //TODO: marke the future as "registered" (must unreg when it fires?) @@ -262,8 +276,9 @@ string label var err = FdbNative.FutureSetCallback(handle, GlobalCallback, cookie); if (!Fdb.Success(err)) - { - throw Fdb.MapToException(err); + { // the callback will not fire, so we have to abort the future immediately + future.TrySetException(Fdb.MapToException(err)); + return future.Task; } mustDispose = false; return future.Task; @@ -273,17 +288,20 @@ string label if (future != null) { future.TrySetException(e); - if (cookie != IntPtr.Zero) - { - lock (m_futures) - { - m_futures.Remove(cookie); - } - } return future.Task; } throw; } + finally + { + if (mustDispose && cookie != IntPtr.Zero) + { // make sure that we never leak a failed future ! + lock (m_futures) + { + m_futures.Remove(cookie); + } + } + } } /// Add a new future handle to this context @@ -303,11 +321,13 @@ protected Task RegisterFutures( string label ) { + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + FdbFutureArray future = null; IntPtr cookie = IntPtr.Zero; try { - uint futureId = (uint)Interlocked.Increment(ref m_localCookieCounter); + uint futureId = (uint) Interlocked.Increment(ref m_localCookieCounter); cookie = MakeCallbackCookie(m_contextId, futureId); // make a copy because we may diverge from the caller if we partially fail to register the callbacks below @@ -315,7 +335,24 @@ string label handles.CopyTo(tmp, 0); future = new FdbFutureArray(tmp, selector, state, cookie, label); - //TODO: we could check if all handles are already completed/failed? + // check the case where all futures are already ready (served from cache?) + bool ready = true; + foreach (var handle in tmp) + { + if (!FdbNative.FutureIsReady(handle)) + { + ready = false; + break; + } + } + if (ready) + { + Debug.WriteLine("FutureArray.{0} [{1}] already completed!", label, tmp.Length); + cookie = IntPtr.Zero; + mustDispose = false; + future.OnReady(); + return future.Task; + } lock (m_futures) { @@ -323,6 +360,16 @@ string label m_futures[cookie] = future; } + if (ct.CanBeCanceled) + { + future.m_ctr = RegisterForCancellation(future, ct); + if (future.Task.IsCompleted) + { // cancellation ran inline + future.TrySetCanceled(); + return future.Task; + } + } + for (int i = 0; i < handles.Length; i++) { var err = FdbNative.FutureSetCallback(handles[i], GlobalCallback, cookie); @@ -354,19 +401,97 @@ string label if (future != null) { future.TrySetException(e); - if (cookie != IntPtr.Zero) - { - lock (m_futures) - { - m_futures.Remove(cookie); - } - } return future.Task; } throw; } + finally + { + if (mustDispose && cookie != IntPtr.Zero) + { // make sure that we never leak a failed future ! + lock (m_futures) + { + m_futures.Remove(cookie); + } + } + + } } + internal static CancellationTokenRegistration RegisterForCancellation(IFdbFuture future, CancellationToken cancellationToken) + { + //note: if the token is already cancelled, the callback handler will run inline and any exception would bubble up here + //=> this is not a problem because the ctor already has a try/catch that will clean up everything + return cancellationToken.Register( + (_state) => { CancellationHandler(_state); }, + future, + false + ); + } + + private static void CancellationHandler(object state) + { + var future = (IFdbFuture)state; + Contract.Assert(state != null); +#if DEBUG_FUTURES + Debug.WriteLine("Future<" + typeof(T).Name + ">.Cancel(0x" + future.m_handle.Handle.ToString("x") + ") was called on thread #" + Thread.CurrentThread.ManagedThreadId.ToString()); +#endif + future.Cancel(); + } + + internal static void DestroyHandle(ref IntPtr handle) + { + if (handle != IntPtr.Zero) + { + FdbNative.FutureDestroy(handle); + handle = IntPtr.Zero; + } + } + + internal static void DestroyHandles(ref IntPtr[] handles) + { + if (handles != null) + { + foreach (var handle in handles) + { + if (handle != IntPtr.Zero) FdbNative.FutureDestroy(handle); + } + handles = null; + } + } + + internal const int CATEGORY_SUCCESS = 0; + internal const int CATEGORY_RETRYABLE = 1; + internal const int CATEGORY_CANCELLED = 2; + internal const int CATEGORY_FAILURE = 3; + + internal static int ClassifyErrorSeverity(FdbError error) + { + switch (error) + { + case FdbError.Success: + { + return CATEGORY_SUCCESS; + } + case FdbError.PastVersion: + case FdbError.FutureVersion: + case FdbError.TimedOut: + case FdbError.TooManyWatches: + { + return CATEGORY_RETRYABLE; + } + + case FdbError.OperationCancelled: + { + return CATEGORY_CANCELLED; + } + + default: + { + return CATEGORY_FAILURE; + } + } + } } internal class FdbFutureContext : FdbFutureContext @@ -472,6 +597,7 @@ protected Task StartNewFutures( throw; } } + } } diff --git a/FoundationDB.Client/Native/FdbFutureSingle.cs b/FoundationDB.Client/Native/FdbFutureSingle.cs index 0bb0ff4ee..c6b918701 100644 --- a/FoundationDB.Client/Native/FdbFutureSingle.cs +++ b/FoundationDB.Client/Native/FdbFutureSingle.cs @@ -29,6 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #undef DEBUG_FUTURES using System.Diagnostics; +using System.Security.Policy; namespace FoundationDB.Client.Native { @@ -50,19 +51,16 @@ internal sealed class FdbFutureSingle : FdbFuture /// Lambda used to extract the result of this FDBFuture private readonly Func m_resultSelector; - private readonly object m_state; - #endregion internal FdbFutureSingle(IntPtr handle, [NotNull] Func selector, object state, IntPtr cookie, string label) - : base(cookie, label) + : base(cookie, label, state) { if (handle == IntPtr.Zero) throw new ArgumentException("Invalid future handle", "handle"); if (selector == null) throw new ArgumentNullException("selector"); m_handle = handle; m_resultSelector = selector; - m_state = state; } public override bool Visit(IntPtr handle) @@ -72,12 +70,9 @@ public override bool Visit(IntPtr handle) } [HandleProcessCorruptedStateExceptions] // to be able to handle Access Violations and terminate the process - public override void OnFired() + public override void OnReady() { - Debug.WriteLine("Future{0}<{1}>.OnFired(0x{2})", this.Label, typeof(T).Name, m_handle.ToString("X8")); - - var handle = Interlocked.Exchange(ref m_handle, IntPtr.Zero); - if (handle == IntPtr.Zero) return; // already disposed? + IntPtr handle = IntPtr.Zero; //README: // - This callback will fire either from the ThreadPool (async ops) or inline form the ctor of the future (non-async ops, or ops that where served from some cache). @@ -86,63 +81,72 @@ public override void OnFired() try { - T result = default(T); - FdbError code; - Exception error = null; - try + handle = Interlocked.Exchange(ref m_handle, IntPtr.Zero); + if (handle == IntPtr.Zero) return; // already disposed? + + Debug.WriteLine("FutureSingle.{0}<{1}>.OnReady(0x{2})", this.Label, typeof(T).Name, handle.ToString("X8")); + + if (this.Task.IsCompleted) + { // task has already been handled by someone else + return; + } + + var result = default(T); + var error = default(Exception); + + var code = FdbNative.FutureGetError(handle); + if (code == FdbError.Success) { - if (this.Task.IsCompleted) - { // task has already been handled by someone else - return; + try + { + result = m_resultSelector(handle, this.Task.AsyncState); } + catch (AccessViolationException e) + { // trouble in paradise! + + Debug.WriteLine("EPIC FAIL: " + e.ToString()); + + // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) + if (Debugger.IsAttached) Debugger.Break(); - code = FdbNative.FutureGetError(handle); - if (code == FdbError.Success) + Environment.FailFast("FIXME: FDB done goofed!", e); + } + catch (Exception e) { - try - { - result = m_resultSelector(handle, m_state); - } - catch (AccessViolationException e) - { // trouble in paradise! - - Debug.WriteLine("EPIC FAIL: " + e.ToString()); - - // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) - if (Debugger.IsAttached) Debugger.Break(); - - Environment.FailFast("FIXME: FDB done goofed!", e); - } - catch (Exception e) - { - Debug.WriteLine("FAIL: " + e.ToString()); - code = FdbError.InternalError; - error = e; - } + Debug.WriteLine("FAIL: " + e.ToString()); + code = FdbError.InternalError; + error = e; } } - finally - { - FdbNative.FutureDestroy(handle); - } + + // since continuations may fire inline, make sure to release all the memory used by this handle first + FdbFutureContext.DestroyHandle(ref handle); if (code == FdbError.Success) { - TrySetResult(result); - } - else if (code == FdbError.OperationCancelled || code == FdbError.TransactionCancelled) - { - TrySetCanceled(); + PublishResult(result); } else { - TrySetException(error ?? Fdb.MapToException(code)); + PublishError(error, code); } } catch (Exception e) { // we must not blow up the TP or the parent, so make sure to propagate all exceptions to the task TrySetException(e); } + finally + { + if (handle != IntPtr.Zero) FdbFutureContext.DestroyHandle(ref handle); + GC.KeepAlive(this); + } + } + + protected override void OnCancel() + { + IntPtr handle = Volatile.Read(ref m_handle); + //TODO: we probably need locking to prevent concurrent destroy and cancel calls + if (handle != IntPtr.Zero) FdbNative.FutureCancel(handle); } } diff --git a/FoundationDB.Client/Native/IFdbFuture.cs b/FoundationDB.Client/Native/IFdbFuture.cs index 317037938..7ec3fc921 100644 --- a/FoundationDB.Client/Native/IFdbFuture.cs +++ b/FoundationDB.Client/Native/IFdbFuture.cs @@ -41,13 +41,16 @@ internal interface IFdbFuture /// Label of the future (usually the name of the operation) string Label { get; } + /// Cancel the future, if it hasen't completed yet + void Cancel(); + /// Test if this was the last pending handle for this future, or not /// Handle that completed - /// True if this was the last handle and OnFired() can be called, or False if more handles need to fire first. + /// True if this was the last handle and can be called, or False if more handles need to fire first. bool Visit(IntPtr handle); /// Called when all handles tracked by this future have fired - void OnFired(); + void OnReady(); } } From 988df11a68c6687446d8a15254bf03481f7e7290 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 11 Feb 2015 14:23:33 +0100 Subject: [PATCH 03/13] Moved future related files into a subfolder --- FoundationDB.Client/FoundationDB.Client.csproj | 10 +++++----- FoundationDB.Client/Native/{ => Futures}/FdbFuture.cs | 0 .../Native/{ => Futures}/FdbFutureArray.cs | 0 .../Native/{ => Futures}/FdbFutureContext.cs | 0 .../Native/{ => Futures}/FdbFutureSingle.cs | 0 FoundationDB.Client/Native/{ => Futures}/IFdbFuture.cs | 0 6 files changed, 5 insertions(+), 5 deletions(-) rename FoundationDB.Client/Native/{ => Futures}/FdbFuture.cs (100%) rename FoundationDB.Client/Native/{ => Futures}/FdbFutureArray.cs (100%) rename FoundationDB.Client/Native/{ => Futures}/FdbFutureContext.cs (100%) rename FoundationDB.Client/Native/{ => Futures}/FdbFutureSingle.cs (100%) rename FoundationDB.Client/Native/{ => Futures}/IFdbFuture.cs (100%) diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 7fdf19767..1b8165770 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -73,8 +73,8 @@ - - + + @@ -111,8 +111,8 @@ - - + + @@ -171,7 +171,7 @@ - + diff --git a/FoundationDB.Client/Native/FdbFuture.cs b/FoundationDB.Client/Native/Futures/FdbFuture.cs similarity index 100% rename from FoundationDB.Client/Native/FdbFuture.cs rename to FoundationDB.Client/Native/Futures/FdbFuture.cs diff --git a/FoundationDB.Client/Native/FdbFutureArray.cs b/FoundationDB.Client/Native/Futures/FdbFutureArray.cs similarity index 100% rename from FoundationDB.Client/Native/FdbFutureArray.cs rename to FoundationDB.Client/Native/Futures/FdbFutureArray.cs diff --git a/FoundationDB.Client/Native/FdbFutureContext.cs b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs similarity index 100% rename from FoundationDB.Client/Native/FdbFutureContext.cs rename to FoundationDB.Client/Native/Futures/FdbFutureContext.cs diff --git a/FoundationDB.Client/Native/FdbFutureSingle.cs b/FoundationDB.Client/Native/Futures/FdbFutureSingle.cs similarity index 100% rename from FoundationDB.Client/Native/FdbFutureSingle.cs rename to FoundationDB.Client/Native/Futures/FdbFutureSingle.cs diff --git a/FoundationDB.Client/Native/IFdbFuture.cs b/FoundationDB.Client/Native/Futures/IFdbFuture.cs similarity index 100% rename from FoundationDB.Client/Native/IFdbFuture.cs rename to FoundationDB.Client/Native/Futures/IFdbFuture.cs From 6da27bdf311086f760cf79cb137b21528c7491a2 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 12 Feb 2015 16:34:11 +0100 Subject: [PATCH 04/13] Future handles are now back to an IntPtr so that can't be compared with null --- FoundationDB.Client/Native/FdbNative.cs | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 50f8b069f..222c6816c 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -431,7 +431,7 @@ public static FdbError StopNetwork() public static IntPtr CreateCluster(string path) { var future = NativeMethods.fdb_create_cluster(path); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_create_cluster(" + path + ") => 0x" + future.ToString("x")); #endif @@ -492,7 +492,7 @@ public static void DatabaseDestroy(IntPtr handle) public static IntPtr ClusterCreateDatabase(ClusterHandle cluster, string name) { var future = NativeMethods.fdb_cluster_create_database(cluster, name, name == null ? 0 : name.Length); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_cluster_create_database(0x" + cluster.Handle.ToString("x") + ", name: '" + name + "') => 0x" + cluster.Handle.ToString("x")); #endif @@ -528,7 +528,7 @@ public static FdbError DatabaseCreateTransaction(DatabaseHandle database, out Tr public static IntPtr TransactionCommit(TransactionHandle transaction) { var future = NativeMethods.fdb_transaction_commit(transaction); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_commit(0x" + transaction.Handle.ToString("x") + ") => 0x" + future.ToString("x")); #endif @@ -542,7 +542,7 @@ public static IntPtr TransactionWatch(TransactionHandle transaction, Slice key) fixed (byte* ptrKey = key.Array) { var future = NativeMethods.fdb_transaction_watch(transaction, ptrKey + key.Offset, key.Count); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_watch(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "') => 0x" + future.ToString("x")); #endif @@ -553,7 +553,7 @@ public static IntPtr TransactionWatch(TransactionHandle transaction, Slice key) public static IntPtr TransactionOnError(TransactionHandle transaction, FdbError errorCode) { var future = NativeMethods.fdb_transaction_on_error(transaction, errorCode); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_on_error(0x" + transaction.Handle.ToString("x") + ", " + errorCode + ") => 0x" + future.ToString("x")); #endif @@ -587,7 +587,7 @@ public static void TransactionSetReadVersion(TransactionHandle transaction, long public static IntPtr TransactionGetReadVersion(TransactionHandle transaction) { var future = NativeMethods.fdb_transaction_get_read_version(transaction); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_get_read_version(0x" + transaction.Handle.ToString("x") + ") => 0x" + future.ToString("x")); #endif @@ -620,7 +620,7 @@ public static IntPtr TransactionGet(TransactionHandle transaction, Slice key, bo fixed (byte* ptrKey = key.Array) { var future = NativeMethods.fdb_transaction_get(transaction, ptrKey + key.Offset, key.Count, snapshot); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_get(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "', snapshot: " + snapshot + ") => 0x" + future.ToString("x")); #endif @@ -638,7 +638,7 @@ public static IntPtr TransactionGetRange(TransactionHandle transaction, FdbKeySe ptrBegin + begin.Key.Offset, begin.Key.Count, begin.OrEqual, begin.Offset, ptrEnd + end.Key.Offset, end.Key.Count, end.OrEqual, end.Offset, limit, targetBytes, mode, iteration, snapshot, reverse); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_get_range(0x" + transaction.Handle.ToString("x") + ", begin: " + begin.PrettyPrint(FdbKey.PrettyPrintMode.Begin) + ", end: " + end.PrettyPrint(FdbKey.PrettyPrintMode.End) + ", " + snapshot + ") => 0x" + future.ToString("x")); #endif @@ -653,7 +653,7 @@ public static IntPtr TransactionGetKey(TransactionHandle transaction, FdbKeySele fixed (byte* ptrKey = selector.Key.Array) { var future = NativeMethods.fdb_transaction_get_key(transaction, ptrKey + selector.Key.Offset, selector.Key.Count, selector.OrEqual, selector.Offset, snapshot); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_get_key(0x" + transaction.Handle.ToString("x") + ", " + selector.ToString() + ", " + snapshot + ") => 0x" + future.ToString("x")); #endif @@ -668,7 +668,7 @@ public static IntPtr TransactionGetAddressesForKey(TransactionHandle transaction fixed (byte* ptrKey = key.Array) { var future = NativeMethods.fdb_transaction_get_addresses_for_key(transaction, ptrKey + key.Offset, key.Count); - Contract.Assert(future != null); + Contract.Assert(future != IntPtr.Zero); #if DEBUG_NATIVE_CALLS Debug.WriteLine("fdb_transaction_get_addresses_for_key(0x" + transaction.Handle.ToString("x") + ", key: '" + FdbKey.Dump(key) + "') => 0x" + future.ToString("x")); #endif From 0b5ae138b03ecb1948769f69e5213780df2baef0 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 12 Feb 2015 16:35:50 +0100 Subject: [PATCH 05/13] Added #if DEBUG_FUTURES to on/off custom logging in all the future callbacks - maybe remove this when we are done? --- FoundationDB.Client/Native/FdbNative.cs | 2 ++ .../Native/Futures/FdbFutureArray.cs | 6 ++++++ .../Native/Futures/FdbFutureContext.cs | 4 +++- .../Native/Futures/FdbFutureSingle.cs | 16 +++++++++++----- 4 files changed, 22 insertions(+), 6 deletions(-) diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 222c6816c..9d89f9ddc 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -353,7 +353,9 @@ public static bool FutureIsReady(IntPtr futureHandle) public static void FutureDestroy(IntPtr futureHandle, [CallerMemberName] string caller = null) { +#if DEBUG_FUTURES Debug.WriteLine("Native.FutureDestroy(0x{0}) from {1}", (object)futureHandle.ToString("X"), caller); +#endif if (futureHandle != IntPtr.Zero) { NativeMethods.fdb_future_destroy(futureHandle); diff --git a/FoundationDB.Client/Native/Futures/FdbFutureArray.cs b/FoundationDB.Client/Native/Futures/FdbFutureArray.cs index 72b0a0c45..60897bf2c 100644 --- a/FoundationDB.Client/Native/Futures/FdbFutureArray.cs +++ b/FoundationDB.Client/Native/Futures/FdbFutureArray.cs @@ -88,7 +88,9 @@ public override void OnReady() handles = Interlocked.Exchange(ref m_handles, null); if (handles == null) return; // already disposed? +#if DEBUG_FUTURES Debug.WriteLine("FutureArray.{0}<{1}[]>.OnReady([{2}])", this.Label, typeof(T).Name, handles.Length); +#endif T[] results = new T[handles.Length]; FdbError code = FdbError.Success; @@ -119,7 +121,9 @@ public override void OnReady() catch (AccessViolationException e) { // trouble in paradise! +#if DEBUG_FUTURES Debug.WriteLine("EPIC FAIL: " + e.ToString()); +#endif // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) if (Debugger.IsAttached) Debugger.Break(); @@ -128,7 +132,9 @@ public override void OnReady() } catch (Exception e) { +#if DEBUG_FUTURES Debug.WriteLine("FAIL: " + e.ToString()); +#endif code = FdbError.InternalError; error = e; break; diff --git a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs index ea7157c28..73cc61825 100644 --- a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs +++ b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs @@ -247,7 +247,9 @@ string label if (FdbNative.FutureIsReady(handle)) { // the result is already computed +#if DEBUG_FUTURES Debug.WriteLine("FutureSingle.{0} 0x{1} already completed!", label, handle.ToString("X")); +#endif cookie = IntPtr.Zero; mustDispose = false; future.OnReady(); @@ -434,7 +436,7 @@ private static void CancellationHandler(object state) var future = (IFdbFuture)state; Contract.Assert(state != null); #if DEBUG_FUTURES - Debug.WriteLine("Future<" + typeof(T).Name + ">.Cancel(0x" + future.m_handle.Handle.ToString("x") + ") was called on thread #" + Thread.CurrentThread.ManagedThreadId.ToString()); + Debug.WriteLine("CancellationHandler for " + future + " was called on thread #" + Thread.CurrentThread.ManagedThreadId.ToString()); #endif future.Cancel(); } diff --git a/FoundationDB.Client/Native/Futures/FdbFutureSingle.cs b/FoundationDB.Client/Native/Futures/FdbFutureSingle.cs index c6b918701..0e177979e 100644 --- a/FoundationDB.Client/Native/Futures/FdbFutureSingle.cs +++ b/FoundationDB.Client/Native/Futures/FdbFutureSingle.cs @@ -26,16 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#undef DEBUG_FUTURES - -using System.Diagnostics; -using System.Security.Policy; namespace FoundationDB.Client.Native { using FoundationDB.Client.Utils; using JetBrains.Annotations; using System; + using System.Diagnostics; using System.Runtime.ExceptionServices; using System.Threading; @@ -65,7 +62,10 @@ internal FdbFutureSingle(IntPtr handle, [NotNull] Func select public override bool Visit(IntPtr handle) { - Contract.Requires(handle == m_handle); +#if DEBUG_FUTURES + Debug.WriteLine("FutureSingle.{0}<{1}>.Visit(0x{2})", this.Label, typeof(T).Name, handle.ToString("X8")); +#endif + Contract.Requires(handle == m_handle, this.Label); return true; } @@ -84,7 +84,9 @@ public override void OnReady() handle = Interlocked.Exchange(ref m_handle, IntPtr.Zero); if (handle == IntPtr.Zero) return; // already disposed? +#if DEBUG_FUTURES Debug.WriteLine("FutureSingle.{0}<{1}>.OnReady(0x{2})", this.Label, typeof(T).Name, handle.ToString("X8")); +#endif if (this.Task.IsCompleted) { // task has already been handled by someone else @@ -104,7 +106,9 @@ public override void OnReady() catch (AccessViolationException e) { // trouble in paradise! +#if DEBUG_FUTURES Debug.WriteLine("EPIC FAIL: " + e.ToString()); +#endif // => THIS IS VERY BAD! We have no choice but to terminate the process immediately, because any new call to any method to the binding may end up freezing the whole process (best case) or sending corrupted data to the cluster (worst case) if (Debugger.IsAttached) Debugger.Break(); @@ -113,7 +117,9 @@ public override void OnReady() } catch (Exception e) { +#if DEBUG_FUTURES Debug.WriteLine("FAIL: " + e.ToString()); +#endif code = FdbError.InternalError; error = e; } From b030e2964db3d095e0ccf9f38a16d12566306347 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 13 Feb 2015 19:18:26 +0100 Subject: [PATCH 06/13] Detect when the main future callback is not called from the network thread, and defer the handling to the thread pool - When callback are invoked inline, we have to return ASAP to the caller (who holds a lock) so we move to the TP - If we have already been moved to the TP, then we can call OnReady inline (already on the TP!) else we defer to the TP at that time. - Use the 32 lower bits as the key in the future dictionary, because IntPtr is not IEquatable and uses a boxing key comparer - Added a "dead" flag to future context, and implemented refcounting down to 0 to remove the context from the global MapToException - Changed the way transaction are cancelled, by waiting for FDB_C to call us back for each pending future (need to map `transaction_cancelled` into a TaskCancelledException for this to work) - Started working on a Watch refactoring --- .../Core/IFdbTransactionHandler.cs | 35 +-- FoundationDB.Client/Fdb.cs | 6 +- FoundationDB.Client/FdbTransaction.cs | 81 ++++-- FoundationDB.Client/Native/FdbNative.cs | 126 +++++++++ .../Native/FdbNativeCluster.cs | 45 +--- .../Native/FdbNativeTransaction.cs | 29 +- .../Native/Futures/FdbFuture.cs | 2 +- .../Native/Futures/FdbFutureContext.cs | 249 ++++++++++++++---- .../Subspaces/Fdb.Directory.cs | 2 +- FoundationDB.Tests/FdbTest.cs | 14 +- FoundationDB.Tests/TransactionFacts.cs | 192 +++++++------- FoundationDb.Client.sln.DotSettings | 4 +- 12 files changed, 538 insertions(+), 247 deletions(-) diff --git a/FoundationDB.Client/Core/IFdbTransactionHandler.cs b/FoundationDB.Client/Core/IFdbTransactionHandler.cs index 49280b240..d26d0682b 100644 --- a/FoundationDB.Client/Core/IFdbTransactionHandler.cs +++ b/FoundationDB.Client/Core/IFdbTransactionHandler.cs @@ -51,7 +51,8 @@ public interface IFdbTransactionHandler : IDisposable void SetOption(FdbTransactionOption option, Slice data); /// Returns this transaction snapshot read version. - Task GetReadVersionAsync(CancellationToken cancellationToken); + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself + Task GetReadVersionAsync(CancellationToken cancellationToken = default(CancellationToken)); /// Retrieves the database version number at which a given transaction was committed. /// CommitAsync() must have been called on this transaction and the resulting task must have completed successfully before this function is callged, or the behavior is undefined. @@ -71,30 +72,30 @@ public interface IFdbTransactionHandler : IDisposable /// Reads a get from the database /// Key to read /// Set to true for snapshot reads - /// + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// - Task GetAsync(Slice key, bool snapshot, CancellationToken cancellationToken); + Task GetAsync(Slice key, bool snapshot, CancellationToken cancellationToken = default(CancellationToken)); /// Reads several values from the database snapshot represented by the current transaction /// Keys to be looked up in the database /// Set to true for snapshot reads - /// Token used to cancel the operation from the outside + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// Task that will return an array of values, or an exception. Each item in the array will contain the value of the key at the same index in , or Slice.Nil if that key does not exist. - Task GetValuesAsync([NotNull] Slice[] keys, bool snapshot, CancellationToken cancellationToken); + Task GetValuesAsync([NotNull] Slice[] keys, bool snapshot, CancellationToken cancellationToken = default(CancellationToken)); /// Resolves a key selector against the keys in the database snapshot represented by the current transaction. /// Key selector to resolve /// Set to true for snapshot reads - /// Token used to cancel the operation from the outside + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// Task that will return the key matching the selector, or an exception - Task GetKeyAsync(FdbKeySelector selector, bool snapshot, CancellationToken cancellationToken); + Task GetKeyAsync(FdbKeySelector selector, bool snapshot, CancellationToken cancellationToken = default(CancellationToken)); /// Resolves several key selectors against the keys in the database snapshot represented by the current transaction. /// Key selectors to resolve /// Set to true for snapshot reads - /// Token used to cancel the operation from the outside + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// Task that will return an array of keys matching the selectors, or an exception - Task GetKeysAsync([NotNull] FdbKeySelector[] selectors, bool snapshot, CancellationToken cancellationToken); + Task GetKeysAsync([NotNull] FdbKeySelector[] selectors, bool snapshot, CancellationToken cancellationToken = default(CancellationToken)); /// Reads all key-value pairs in the database snapshot represented by transaction (potentially limited by Limit, TargetBytes, or Mode) which have a key lexicographically greater than or equal to the key resolved by the begin key selector and lexicographically less than the key resolved by the end key selector. /// key selector defining the beginning of the range @@ -102,15 +103,15 @@ public interface IFdbTransactionHandler : IDisposable /// Optionnal query options (Limit, TargetBytes, Mode, Reverse, ...) /// If streaming mode is FdbStreamingMode.Iterator, this parameter should start at 1 and be incremented by 1 for each successive call while reading this range. In all other cases it is ignored. /// Set to true for snapshot reads - /// Token used to cancel the operation from the outside + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// - Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, [NotNull] FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken); + Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, [NotNull] FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken = default(CancellationToken)); /// Returns a list of public network addresses as strings, one for each of the storage servers responsible for storing and its associated value /// Name of the key whose location is to be queried. - /// Token used to cancel the operation from the outside + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// Task that will return an array of strings, or an exception - Task GetAddressesForKeyAsync(Slice key, CancellationToken cancellationToken); + Task GetAddressesForKeyAsync(Slice key, CancellationToken cancellationToken = default(CancellationToken)); /// Modify the database snapshot represented by transaction to change the given key to have the given value. If the given key was not previously present in the database it is inserted. /// The modification affects the actual database only if transaction is later committed with CommitAsync(). @@ -153,19 +154,19 @@ public interface IFdbTransactionHandler : IDisposable /// The commit may or may not succeed – in particular, if a conflicting transaction previously committed, then the commit must fail in order to preserve transactional isolation. /// If the commit does succeed, the transaction is durably committed to the database and all subsequently started transactions will observe its effects. /// - /// Token used to cancel the operation from the outside + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// Task that succeeds if the transaction was comitted successfully, or fails if the transaction failed to commit. /// As with other client/server databases, in some failure scenarios a client may be unable to determine whether a transaction succeeded. In these cases, CommitAsync() will throw CommitUnknownResult error. The OnErrorAsync() function treats this error as retryable, so retry loops that don’t check for CommitUnknownResult could execute the transaction twice. In these cases, you must consider the idempotence of the transaction. - Task CommitAsync(CancellationToken cancellationToken); + Task CommitAsync(CancellationToken cancellationToken = default(CancellationToken)); /// Implements the recommended retry and backoff behavior for a transaction. /// This function knows which of the error codes generated by other query functions represent temporary error conditions and which represent application errors that should be handled by the application. /// It also implements an exponential backoff strategy to avoid swamping the database cluster with excessive retries when there is a high level of conflict between transactions. /// /// FdbError code thrown by the previous command - /// Token used to cancel the operation from the outside + /// Token used to cancel the operation from the outside, if different than the cancellation token of the transaction itself /// Returns a task that completes if the operation can be safely retried, or that rethrows the original exception if the operation is not retryable. - Task OnErrorAsync(FdbError code, CancellationToken cancellationToken); + Task OnErrorAsync(FdbError code, CancellationToken cancellationToken = default(CancellationToken)); /// Reset transaction to its initial state. /// This is similar to disposing the transaction and recreating a new one. The only state that persists through a transaction reset is that which is related to the backoff logic used by OnErrorAsync() diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index bef6b2545..145928eea 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -523,13 +523,13 @@ internal static async Task CreateClusterInternalAsync(string cluster /// If is anything other than 'DB' /// If the token is cancelled /// Since connections are not pooled, so this method can be costly and should NOT be called every time you need to read or write from the database. Instead, you should open a database instance at the start of your process, and use it a singleton. - public static async Task OpenAsync(string clusterFile, string dbName, FdbSubspace globalSpace, bool readOnly = false, CancellationToken cancellationToken = default(CancellationToken)) + public static Task OpenAsync(string clusterFile, string dbName, FdbSubspace globalSpace, bool readOnly = false, CancellationToken cancellationToken = default(CancellationToken)) { - return await OpenInternalAsync(clusterFile, dbName, globalSpace, readOnly, cancellationToken); + return OpenInternalAsync(clusterFile, dbName, globalSpace, readOnly, cancellationToken); } /// Create a new database handler instance using the specificied cluster file, database name, global subspace and read only settings - internal static async Task OpenInternalAsync(string clusterFile, string dbName, FdbSubspace globalSpace, bool readOnly, CancellationToken cancellationToken) + internal static async Task OpenInternalAsync(string clusterFile, string dbName, FdbSubspace globalSpace, bool readOnly, CancellationToken cancellationToken) { cancellationToken.ThrowIfCancellationRequested(); diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index 22adf2e52..fbeac558b 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -90,6 +90,9 @@ public sealed partial class FdbTransaction : IFdbTransaction, IFdbReadOnlyTransa /// CancellationToken that should be used for all async operations executing inside this transaction private CancellationToken m_cancellation; //PERF: readonly struct + /// Used to cancel the transaction if the parent CTS fires + private CancellationTokenRegistration m_ctr; + #endregion #region Constructors... @@ -108,6 +111,15 @@ internal FdbTransaction(FdbDatabase db, FdbOperationContext context, int id, IFd m_readOnly = (mode & FdbTransactionMode.ReadOnly) != 0; m_handler = handler; + + if (m_cancellation.IsCancellationRequested) + { // already dead? + Cancel(explicitly: false); + } + else + { + m_ctr = m_cancellation.Register(CancellationHandler, this); + } } #endregion @@ -263,7 +275,7 @@ public Task GetReadVersionAsync() // can be called after the transaction has been committed EnsureCanRetry(); - return m_handler.GetReadVersionAsync(m_cancellation); + return m_handler.GetReadVersionAsync(CancellationToken.None); } /// Retrieves the database version number at which a given transaction was committed. @@ -310,7 +322,7 @@ public Task GetAsync(Slice key) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAsync", String.Format("Getting value for '{0}'", key.ToString())); #endif - return m_handler.GetAsync(key, snapshot: false, cancellationToken: m_cancellation); + return m_handler.GetAsync(key, snapshot: false, cancellationToken: CancellationToken.None); } #endregion @@ -333,7 +345,7 @@ public Task GetValuesAsync(Slice[] keys) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetValuesAsync", String.Format("Getting batch of {0} values ...", keys.Length)); #endif - return m_handler.GetValuesAsync(keys, snapshot: false, cancellationToken: m_cancellation); + return m_handler.GetValuesAsync(keys, snapshot: false, cancellationToken: CancellationToken.None); } #endregion @@ -363,7 +375,7 @@ public Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySe // The iteration value is only needed when in iterator mode, but then it should start from 1 if (iteration == 0) iteration = 1; - return m_handler.GetRangeAsync(beginInclusive, endExclusive, options, iteration, snapshot: false, cancellationToken: m_cancellation); + return m_handler.GetRangeAsync(beginInclusive, endExclusive, options, iteration, snapshot: false, cancellationToken: CancellationToken.None); } #endregion @@ -416,7 +428,7 @@ public async Task GetKeyAsync(FdbKeySelector selector) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeyAsync", String.Format("Getting key '{0}'", selector.ToString())); #endif - var key = await m_handler.GetKeyAsync(selector, snapshot: false, cancellationToken: m_cancellation).ConfigureAwait(false); + var key = await m_handler.GetKeyAsync(selector, snapshot: false, cancellationToken: CancellationToken.None).ConfigureAwait(false); // don't forget to truncate keys that would fall outside of the database's globalspace ! return m_database.BoundCheck(key); @@ -444,7 +456,7 @@ public Task GetKeysAsync(FdbKeySelector[] selectors) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeysAsync", String.Format("Getting batch of {0} keys ...", selectors.Length)); #endif - return m_handler.GetKeysAsync(selectors, snapshot: false, cancellationToken: m_cancellation); + return m_handler.GetKeysAsync(selectors, snapshot: false, cancellationToken: CancellationToken.None); } #endregion @@ -634,7 +646,7 @@ public Task GetAddressesForKeyAsync(Slice key) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAddressesForKeyAsync", String.Format("Getting addresses for key '{0}'", FdbKey.Dump(key))); #endif - return m_handler.GetAddressesForKeyAsync(key, cancellationToken: m_cancellation); + return m_handler.GetAddressesForKeyAsync(key, CancellationToken.None); } #endregion @@ -657,7 +669,7 @@ public async Task CommitAsync() //TODO: need a STATE_COMMITTING ? try { - await m_handler.CommitAsync(m_cancellation).ConfigureAwait(false); + await m_handler.CommitAsync(CancellationToken.None).ConfigureAwait(false); if (Interlocked.CompareExchange(ref m_state, STATE_COMMITTED, STATE_READY) == STATE_READY) { @@ -724,7 +736,7 @@ public async Task OnErrorAsync(FdbError code) { EnsureCanRetry(); - await m_handler.OnErrorAsync(code, cancellationToken: m_cancellation).ConfigureAwait(false); + await m_handler.OnErrorAsync(code, CancellationToken.None).ConfigureAwait(false); // If fdb_transaction_on_error succeeds, that means that the transaction has been reset and is usable again var state = this.State; @@ -777,19 +789,47 @@ public void Reset() /// Rollback this transaction, and dispose it. It should not be used after that. public void Cancel() + { + Cancel(explicitly: true); + } + + private void Cancel(bool explicitly) { var state = Interlocked.CompareExchange(ref m_state, STATE_CANCELED, STATE_READY); if (state != STATE_READY) { - switch(state) + if (explicitly) { - case STATE_CANCELED: return; // already the case ! + switch (state) + { + case STATE_CANCELED: + { + return; // already the case! + } + case STATE_COMMITTED: + { + throw new InvalidOperationException("Cannot cancel transaction that has already been committed"); + } + case STATE_FAILED: + { + throw new InvalidOperationException("Cannot cancel transaction because it is in a failed state"); + } + case STATE_DISPOSED: + { + throw new ObjectDisposedException("FdbTransaction", "Cannot cancel transaction because it already has been disposed"); + } + default: + { + throw new InvalidOperationException(String.Format("Cannot cancel transaction because it is in unknown state {0}", state)); + } + } + } - case STATE_COMMITTED: throw new InvalidOperationException("Cannot cancel transaction that has already been committed"); - case STATE_FAILED: throw new InvalidOperationException("Cannot cancel transaction because it is in a failed state"); - case STATE_DISPOSED: throw new ObjectDisposedException("FdbTransaction", "Cannot cancel transaction because it already has been disposed"); - default: throw new InvalidOperationException(String.Format("Cannot cancel transaction because it is in unknown state {0}", state)); + if (state == STATE_CANCELED || state == STATE_DISPOSED) + { // it's too late + return; } + } if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Cancel", "Canceling transaction..."); @@ -799,6 +839,16 @@ public void Cancel() if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Cancel", "Transaction has been canceled"); } + private static readonly Action CancellationHandler = CancellationCallback; + + /// Handler called when the cancellation source of the transaction fires + private static void CancellationCallback(object state) + { + Contract.Requires(state != null); + var trans = (FdbTransaction) state; + trans.Cancel(explicitly: false); + } + #endregion #region IDisposable... @@ -916,6 +966,7 @@ public void Dispose() { try { + m_ctr.Dispose(); this.Database.UnregisterTransaction(this); m_cts.SafeCancelAndDispose(); diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 9d89f9ddc..487afa18f 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -38,6 +38,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY using System.Runtime.ExceptionServices; using System.Runtime.InteropServices; using System.Text; +using System.Threading; +using System.Threading.Tasks; +using FoundationDB.Client.Core; namespace FoundationDB.Client.Native { @@ -889,6 +892,129 @@ public static FdbError TransactionAddConflictRange(TransactionHandle transaction #endregion + #region Global Future Context... + + internal static readonly GlobalNativeContext GlobalContext = new GlobalNativeContext(); + + internal sealed class GlobalNativeContext : FdbFutureContext + { + + public Task CreateClusterAsync(string clusterFile, CancellationToken ct) + { + return RunAsync( + (arg) => FdbNative.CreateCluster((string)arg), + clusterFile, + (h, _) => + { + ClusterHandle cluster; + var err = FdbNative.FutureGetCluster(h, out cluster); + if (err != FdbError.Success) + { + cluster.Dispose(); + throw Fdb.MapToException(err); + } + var handler = new FdbNativeCluster(cluster); + return (IFdbClusterHandler)handler; + }, + null, //unused + ct + ); + } + + public void WatchKeyAsync(ref FdbWatch watch, TransactionHandle handle, Slice key, CancellationToken ct) + { + throw new NotImplementedException(); +#if false + IntPtr h = IntPtr.Zero; + bool mustDispose = true; + try + { + IntPtr cookie = IntPtr.Zero; //TODO!! + + RuntimeHelpers.PrepareConstrainedRegions(); + try + { } + finally + { + h = FdbNative.TransactionWatch(handle, key); + } + Contract.Assert(h != IntPtr.Zero); + if (h == IntPtr.Zero) throw new InvalidOperationException("FIXME: failed to create a watch handle");//TODO: message? + + var f = new FdbWatchFuture(key, cookie, "WatchKeyAsync", null); + watch = new FdbWatch(f, key, Slice.Nil); + + if (FdbNative.FutureIsReady(h)) + { + f.OnReady(); + mustDispose = false; + return; + } + } + finally + { + if (mustDispose && h != IntPtr.Zero) + { + FdbNative.FutureDestroy(h); + } + } +#endif + } + + internal sealed class FdbWatchFuture : FdbFuture + { + private IntPtr m_handle; + + private readonly object m_lock = new object(); + + public FdbWatchFuture(Slice key, IntPtr cookie, string label, object state) + : base(cookie, label, state) + { + this.Key = key; + } + + public Slice Key { get; private set; } + + public override bool Visit(IntPtr handle) + { + Contract.Assert(handle == m_handle || m_handle == IntPtr.Zero); + return true; + } + + protected override void OnCancel() + { + throw new NotImplementedException(); + } + + public override void OnReady() + { + IntPtr handle = IntPtr.Zero; + try + { + handle = Interlocked.Exchange(ref m_handle, IntPtr.Zero); + if (handle == IntPtr.Zero) return; + + var err = FdbNative.FutureGetError(m_handle); + + if (err == FdbError.Success) + { + PublishResult(this.Key); + } + else + { + PublishError(null, err); + } + } + finally + { + if (handle != IntPtr.Zero) FdbNative.FutureDestroy(handle); + } + } + } + + } + +#endregion } } diff --git a/FoundationDB.Client/Native/FdbNativeCluster.cs b/FoundationDB.Client/Native/FdbNativeCluster.cs index 7c5d8e42a..4dfecdde0 100644 --- a/FoundationDB.Client/Native/FdbNativeCluster.cs +++ b/FoundationDB.Client/Native/FdbNativeCluster.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Client.Native using FoundationDB.Client.Utils; using System; using System.Diagnostics; + using System.Runtime.CompilerServices; using System.Threading; using System.Threading.Tasks; @@ -46,49 +47,9 @@ public FdbNativeCluster(ClusterHandle handle) { } - private static readonly GlobalNativeContext GlobalContext = new GlobalNativeContext(); - - private sealed class GlobalNativeContext : FdbFutureContext - { - - public Task CreateClusterAsync(string clusterFile, CancellationToken ct) - { - IntPtr handle = IntPtr.Zero; - bool dead = true; - try - { - return RegisterFuture( - FdbNative.CreateCluster(clusterFile), - ref dead, - (h, state) => - { - ClusterHandle cluster; - var err = FdbNative.FutureGetCluster(h, out cluster); - if (err != FdbError.Success) - { - cluster.Dispose(); - throw Fdb.MapToException(err); - } - var handler = new FdbNativeCluster(cluster); - return (IFdbClusterHandler) handler; - }, - null, - ct, - "CreateClusterAsync" - ); - } - finally - { - if (handle != IntPtr.Zero && dead) FdbNative.FutureDestroy(handle); - } - } - - } - - public static Task CreateClusterAsync(string clusterFile, CancellationToken cancellationToken) { - return GlobalContext.CreateClusterAsync(clusterFile, cancellationToken); + return FdbNative.GlobalContext.CreateClusterAsync(clusterFile, cancellationToken); } public bool IsInvalid { get { return m_handle.IsInvalid; } } @@ -119,7 +80,7 @@ public Task OpenDatabaseAsync(string databaseName, Cancella { if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); - return StartNewFuture( + return RunAsync( (handle, state) => FdbNative.ClusterCreateDatabase(handle, state), databaseName, (h, state) => diff --git a/FoundationDB.Client/Native/FdbNativeTransaction.cs b/FoundationDB.Client/Native/FdbNativeTransaction.cs index 43f5ad3a7..8cfe68bc0 100644 --- a/FoundationDB.Client/Native/FdbNativeTransaction.cs +++ b/FoundationDB.Client/Native/FdbNativeTransaction.cs @@ -109,7 +109,7 @@ public void SetOption(FdbTransactionOption option, Slice data) public Task GetReadVersionAsync(CancellationToken cancellationToken) { - return StartNewFuture( + return RunAsync( (handle, state) => FdbNative.TransactionGetReadVersion(handle), default(object), (future, state) => @@ -151,7 +151,7 @@ private static Slice GetValueResultBytes(IntPtr h) public Task GetAsync(Slice key, bool snapshot, CancellationToken cancellationToken) { - return StartNewFuture( + return RunAsync( (handle, state) => FdbNative.TransactionGet(handle, state.Item1, state.Item2), FdbTuple.Create(key, snapshot), (future, state) => GetValueResultBytes(future), @@ -166,7 +166,7 @@ public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationTok if (keys.Length == 0) return Task.FromResult(Slice.EmptySliceArray); - return StartNewFutures( + return RunAsync( keys.Length, (handle, state, futures) => { @@ -209,7 +209,7 @@ public Task GetRangeAsync(FdbKeySelector begin, FdbKeySelector en bool reversed = options.Reverse ?? false; - return StartNewFuture( + return RunAsync( (handle, _) => FdbNative.TransactionGetRange(handle, begin, end, options.Limit ?? 0, options.TargetBytes ?? 0, options.Mode ?? FdbStreamingMode.Iterator, iteration, snapshot, reversed), default(object), //TODO: pass options & co? (future, state) => @@ -239,7 +239,7 @@ private static Slice GetKeyResult(IntPtr h) public Task GetKeyAsync(FdbKeySelector selector, bool snapshot, CancellationToken cancellationToken) { - return StartNewFuture( + return RunAsync( (handle, state) => FdbNative.TransactionGetKey(handle, state.Item1, state.Item2), FdbTuple.Create(selector, snapshot), (future, state) => GetKeyResult(future), @@ -254,7 +254,7 @@ public Task GetKeysAsync(FdbKeySelector[] selectors, bool snapshot, Can if (selectors.Length == 0) return Task.FromResult(Slice.EmptySliceArray); - return StartNewFutures( + return RunAsync( selectors.Length, (handle, state, futures) => { @@ -331,7 +331,7 @@ private static string[] GetStringArrayResult(IntPtr h) public Task GetAddressesForKeyAsync(Slice key, CancellationToken cancellationToken) { - return StartNewFuture( + return RunAsync( (handle, state) => FdbNative.TransactionGetAddressesForKey(handle, state), key, (future, state) => GetStringArrayResult(future), @@ -346,13 +346,12 @@ public Task GetAddressesForKeyAsync(Slice key, CancellationToken cance public FdbWatch Watch(Slice key, CancellationToken cancellationToken) { + // a Watch will outlive the transaction, so we can attach it to the current FutureContext (which will be disposed once the transaction goes away) + // => we will store at them to the GlobalContext + + + throw new NotImplementedException("FIXME: Future refactoring in progress! I owe you a beer (*) if I ever forget to remove this before committing! (*: if you come get it in person!)"); - //var future = FdbNative.TransactionWatch(m_handle, key); - //return new FdbWatch( - // FdbFuture.FromHandle(future, (h) => key, cancellationToken), - // key, - // Slice.Nil - //); } #endregion @@ -379,7 +378,7 @@ public long GetCommittedVersion() /// As with other client/server databases, in some failure scenarios a client may be unable to determine whether a transaction succeeded. In these cases, CommitAsync() will throw CommitUnknownResult error. The OnErrorAsync() function treats this error as retryable, so retry loops that don’t check for CommitUnknownResult could execute the transaction twice. In these cases, you must consider the idempotence of the transaction. public Task CommitAsync(CancellationToken cancellationToken) { - return StartNewFuture( + return RunAsync( (handle, state) => FdbNative.TransactionCommit(handle), default(object), (future, state) => state, @@ -390,7 +389,7 @@ public Task CommitAsync(CancellationToken cancellationToken) public Task OnErrorAsync(FdbError code, CancellationToken cancellationToken) { - return StartNewFuture( + return RunAsync( (handle, state) => FdbNative.TransactionOnError(handle, state), code, (h, state) => diff --git a/FoundationDB.Client/Native/Futures/FdbFuture.cs b/FoundationDB.Client/Native/Futures/FdbFuture.cs index 4a0387d7e..a72053778 100644 --- a/FoundationDB.Client/Native/Futures/FdbFuture.cs +++ b/FoundationDB.Client/Native/Futures/FdbFuture.cs @@ -99,7 +99,7 @@ protected void PublishResult(T result) TrySetResult(result); } - protected void PublishError(Exception error, FdbError code) + protected void PublishError(Exception error, FdbError code) { if (error != null) { diff --git a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs index 73cc61825..f3e5eda2b 100644 --- a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs +++ b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs @@ -27,7 +27,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #endregion // enable this to capture the stacktrace of the ctor, when troubleshooting leaked transaction handles -#undef CAPTURE_STACKTRACES +//#define CAPTURE_STACKTRACES using FoundationDB.Async; @@ -75,23 +75,67 @@ private static IntPtr MakeCallbackCookie(uint contextId, uint futureId) return new IntPtr((long)cookie); } + private static uint GetContextIdFromCookie(IntPtr cookie) + { + return (uint) (((ulong) cookie.ToInt64() >> CONTEXT_COOKIE_SHIFT) & CONTEXT_COOKIE_MASK); + } + + private static uint GetFutureIdFromCookie(IntPtr cookie) + { + return (uint)(((ulong)cookie.ToInt64() >> FUTURE_COOKIE_SHIFT) & FUTURE_COOKIE_MASK); + } + + /// Delegate that will be called by fdb_c to notify us that a future as completed + /// It is important to make sure that this delegate will NOT be garbaged collected before the last future callback has fired! private static readonly FdbNative.FdbFutureCallback GlobalCallback = FutureCallbackHandler; private static void FutureCallbackHandler(IntPtr handle, IntPtr cookie) { // cookie is the value that will help us find the corresponding context (upper 32 bits) and future within this context (lower 32 bits) that matches with this future handle. - //note: this callback can be called either: - // - from the thread that is constructing the Future, if the future was already completed (called inline) - // - from the network thread, when the future completed asynchronously +#if DEBUG_FUTURES + Debug.WriteLine("FutureCallbackHandler(0x{0}, {1:X8} | {2:X8}) called from {3} [{4}]", handle.ToString("X"), cookie.ToInt64() >> 32, cookie.ToInt64() & uint.MaxValue, Thread.CurrentThread.ManagedThreadId, Thread.CurrentThread.Name); +#endif + bool fromNetworkThread = Fdb.IsNetworkThread; - Debug.WriteLine("FutureCallbackHandler(0x{0}, {1:X8} | {2:X8}) called", handle.ToString("X"), cookie.ToInt64() >> 32, cookie.ToInt64() & uint.MaxValue); + if (!fromNetworkThread) + { // most probably, we have been called inline from fdb_future_set_callback + // => The caller is holding a lock, so we have to defer to the ThreadPool and return as soon as possible! + try + { + ThreadPool.UnsafeQueueUserWorkItem( + (state) => + { + var args = (Tuple) state; + ProcessFutureCallback(args.Item1, args.Item2, false); + }, + Tuple.Create(handle, cookie) + ); + return; + } + catch (Exception) + { // unable to defer to the TP? + // we can't rethrow the exception if FDB_C is calling us (it won't know about it), + // so we will continue running inline. Hopefully this should never happen. + + // => eat the exception and continue + } + } + + ProcessFutureCallback(handle, cookie, fromNetworkThread); + } + private static void ProcessFutureCallback(IntPtr handle, IntPtr cookie, bool fromNetworkThread) + { +#if DEBUG_FUTURES + Debug.WriteLine("ProcessFutureCallback(0x{0}, {1:X8} | {2:X8}, {3}) called from {4} [{5}]", handle.ToString("X"), cookie.ToInt64() >> 32, cookie.ToInt64() & uint.MaxValue, fromNetworkThread, Thread.CurrentThread.ManagedThreadId, Thread.CurrentThread.Name); +#endif + // we are called by FDB_C, from the thread that runs the Event Loop bool keepAlive = false; try { - - uint contextId = (uint) (((ulong) cookie.ToInt64() >> CONTEXT_COOKIE_SHIFT) & CONTEXT_COOKIE_MASK); + // extract the upper 32 bits which contain the ID of the corresponding future context + uint contextId = GetContextIdFromCookie(cookie); FdbFutureContext context; lock (s_contexts) // there will only be contentions on this lock if other a lot of threads are creating new contexts (ie: new transactions) @@ -101,8 +145,19 @@ private static void FutureCallbackHandler(IntPtr handle, IntPtr cookie) if (context != null) { + //TODO: if the context is marked as "dead" we need to refcount the pending futures down to 0, and then remove the context from the list + Contract.Assert(context.m_contextId == contextId); - keepAlive = context.OnFutureReady(handle, cookie); + bool purgeContext; + keepAlive = context.OnFutureReady(handle, cookie, fromNetworkThread, out purgeContext); + + if (purgeContext) + { // the context was disposed and saw the last pending future going by, we have to remove it from the list + lock (s_contexts) + { + s_contexts.Remove(contextId); + } + } } } finally @@ -113,6 +168,12 @@ private static void FutureCallbackHandler(IntPtr handle, IntPtr cookie) #endregion + private const int STATE_DEFAULT = 0; + + private const int STATE_DEAD = 1; + + // this flag must only be used under the lock + private int m_flags; /// Cookie for this context /// Makes the 32-bits upper bits of the future callback parameter @@ -123,16 +184,17 @@ private static void FutureCallbackHandler(IntPtr handle, IntPtr cookie) /// Dictionary used to store all the pending Futures for this context /// All methods should take a lock on this instance before manipulating the state - private readonly Dictionary m_futures = new Dictionary(); + private readonly Dictionary m_futures = new Dictionary(); #if CAPTURE_STACKTRACES - private StackTrace m_stackTrace; + private readonly StackTrace m_stackTrace; #endif #region Constructors... protected FdbFutureContext() { + //REVIEW: is this a good idea to do this in the constructor? (we could start observing a context that hasn't been fully constructed yet lock (s_contexts) { s_contexts[m_contextId] = this; @@ -142,18 +204,23 @@ protected FdbFutureContext() #endif } +#if NOT_NEEDED //REVIEW: do we really need a destructor ? The handle is a SafeHandle, and will take care of itself... ~FdbFutureContext() { + if (!AppDomain.CurrentDomain.IsFinalizingForUnload()) + { #if CAPTURE_STACKTRACES - Trace.WriteLine("A transaction handle (" + m_handle + ", " + m_payloadBytes + " bytes written) was leaked by " + m_stackTrace); + Debug.WriteLine("A future context ({0}) was leaked by {1}", this, m_stackTrace); #endif #if DEBUG - // If you break here, that means that a native transaction handler was leaked by a FdbTransaction instance (or that the transaction instance was leaked) - if (Debugger.IsAttached) Debugger.Break(); + // If you break here, that means that a native transaction handler was leaked by a FdbTransaction instance (or that the transaction instance was leaked) + if (Debugger.IsAttached) Debugger.Break(); #endif - Dispose(false); + Dispose(false); + } } +#endif #endregion @@ -167,53 +234,83 @@ public void Dispose() protected virtual void Dispose(bool disposing) { - //Debug.WriteLine("Disposified from " + new StackTrace()); if (disposing) { - lock (s_contexts) + // + +#if DEBUG_FUTURES + Debug.WriteLine("Disposing context {0}#{1} with {2} pending future(s) ({3} total)", this.GetType().Name, m_contextId, m_futures.Count, m_localCookieCounter); +#endif + bool purge; + lock (m_futures) { - Debug.WriteLine("Disposed context {0}#{1} with {2} pending future(s) ({3} total)", this.GetType().Name, m_contextId, m_futures.Count, m_localCookieCounter); - s_contexts.Remove(m_contextId); - foreach (var ctx in s_contexts) + if (m_flags == STATE_DEAD) + { // already dead! + return; + } + m_flags = STATE_DEAD; + purge = m_futures.Count == 0; + } + + if (purge) + { // no pending futures, we can remove ourselves from the global list + lock (s_contexts) { - Debug.WriteLine("- {0}#{1} : {2} ({3})", ctx.Value.GetType().Name, ctx.Key, ctx.Value.m_futures.Count, ctx.Value.m_localCookieCounter); + s_contexts.Remove(m_contextId); +#if DEBUG_FUTURES + Debug.WriteLine("Dumping all remaining contexts: {0}", s_contexts.Count); + foreach (var ctx in s_contexts) + { + Debug.WriteLine("- {0}#{1} : {2} ({3})", ctx.Value.GetType().Name, ctx.Key, ctx.Value.m_futures.Count, ctx.Value.m_localCookieCounter); + } +#endif } } + //else: we have to wait for all callbacks to fire. The last one will remove this context from the global list } } #endregion /// A callback has fire for a future handled by this context - /// - /// - private bool OnFutureReady(IntPtr handle, IntPtr cookie) + private bool OnFutureReady(IntPtr handle, IntPtr cookie, bool fromNetworkThread, out bool purgeContext) { + uint futureId = GetFutureIdFromCookie(cookie); + + purgeContext = false; IFdbFuture future; lock (m_futures) { - m_futures.TryGetValue(cookie, out future); + if (m_flags == STATE_DEAD) + { // we are just waiting for all callbacks to fire + m_futures.Remove(futureId); + purgeContext = m_futures.Count == 0; + return false; + } + + m_futures.TryGetValue(futureId, out future); } if (future != null && future.Cookie == cookie) { if (future.Visit(handle)) { // future is ready to process all the results - ThreadPool.UnsafeQueueUserWorkItem( - (state) => - { - ((IFdbFuture)state).OnReady(); - //TODO: if it fails, maybe we should remove it from m_futures? - }, - future - ); + + if (fromNetworkThread) + { + ThreadPool.UnsafeQueueUserWorkItem((state) => ((IFdbFuture)state).OnReady(), future); + //TODO: what happens if TP.UQUWI() fails? + } + else + { + future.OnReady(); + } } // else: expecting more handles // handles will be destroyed when the future completes return true; } - return false; } @@ -222,13 +319,14 @@ private bool OnFutureReady(IntPtr handle, IntPtr cookie) /// Handle of the newly created future /// Flag set to true if the future must be disposed by the caller (in case of error), or false if the future will be disposed by some other thread. /// Method called when the future completes successfully + /// State that will be passed as the second argument to /// TODO: remove this? /// Type of future (name of the caller) /// protected Task RegisterFuture( IntPtr handle, ref bool mustDispose, - Func selector, + [NotNull] Func selector, object state, CancellationToken ct, string label @@ -238,9 +336,10 @@ string label FdbFutureSingle future = null; IntPtr cookie = IntPtr.Zero; + uint futureId = (uint)Interlocked.Increment(ref m_localCookieCounter); + try { - uint futureId = (uint) Interlocked.Increment(ref m_localCookieCounter); cookie = MakeCallbackCookie(m_contextId, futureId); future = new FdbFutureSingle(handle, selector, state, cookie, label); @@ -273,7 +372,7 @@ string label lock (m_futures) { //TODO: marke the future as "registered" (must unreg when it fires?) - m_futures[cookie] = future; + m_futures[futureId] = future; } var err = FdbNative.FutureSetCallback(handle, GlobalCallback, cookie); @@ -300,7 +399,7 @@ string label { // make sure that we never leak a failed future ! lock (m_futures) { - m_futures.Remove(cookie); + m_futures.Remove(futureId); } } } @@ -311,13 +410,14 @@ string label /// Handles of the newly created future /// Flag set to true if the future must be disposed by the caller (in case of error), or false if the future will be disposed by some other thread. /// Method called when the future completes successfully + /// State that will be passed as the second argument to /// TODO: remove this? /// Type of future (name of the caller) /// protected Task RegisterFutures( - IntPtr[] handles, + [NotNull] IntPtr[] handles, ref bool mustDispose, - Func selector, + [NotNull] Func selector, object state, CancellationToken ct, string label @@ -327,9 +427,9 @@ string label FdbFutureArray future = null; IntPtr cookie = IntPtr.Zero; + uint futureId = (uint) Interlocked.Increment(ref m_localCookieCounter); try { - uint futureId = (uint) Interlocked.Increment(ref m_localCookieCounter); cookie = MakeCallbackCookie(m_contextId, futureId); // make a copy because we may diverge from the caller if we partially fail to register the callbacks below @@ -359,7 +459,7 @@ string label lock (m_futures) { //TODO: mark the future as "registered" (must unreg when it fires?) - m_futures[cookie] = future; + m_futures[futureId] = future; } if (ct.CanBeCanceled) @@ -384,7 +484,7 @@ string label // mute this future lock (m_futures) { - m_futures.Remove(cookie); + m_futures.Remove(futureId); //TODO: mark the future as "unregistered" for (int j = i + 1; j < handles.Length; j++) { @@ -413,13 +513,56 @@ string label { // make sure that we never leak a failed future ! lock (m_futures) { - m_futures.Remove(cookie); + m_futures.Remove(futureId); } } } } + /// Start a new async operation + /// Result of the operation + /// Lambda called to produce the future handle + /// Argument passed to . It will not be used after the handle has been constructed + /// Lambda called once the future completes (successfully) + /// State object passed to . It will be stored in the future has long as it is active. + /// Optional cancellation token used to cancel the task from an external source. + /// Optional label, used for logging and troubleshooting purpose (by default the name of the caller) + /// + protected Task RunAsync( + [NotNull] Func generator, + object argument, + [NotNull] Func selector, + object state, + CancellationToken ct, + [CallerMemberName] string label = null + ) + { + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + + bool mustDispose = true; + IntPtr h = IntPtr.Zero; + try + { + RuntimeHelpers.PrepareConstrainedRegions(); + try + { } + finally + { + h = generator(argument); + } + + return RegisterFuture(h, ref mustDispose, selector, state, ct, label); + } + finally + { + if (mustDispose && h != IntPtr.Zero) + { + FdbNative.FutureDestroy(h); + } + } + } + internal static CancellationTokenRegistration RegisterForCancellation(IFdbFuture future, CancellationToken cancellationToken) { //note: if the token is already cancelled, the callback handler will run inline and any exception would bubble up here @@ -477,13 +620,14 @@ internal static int ClassifyErrorSeverity(FdbError error) } case FdbError.PastVersion: case FdbError.FutureVersion: - case FdbError.TimedOut: - case FdbError.TooManyWatches: + case FdbError.NotCommitted: + case FdbError.CommitUnknownResult: { return CATEGORY_RETRYABLE; } - case FdbError.OperationCancelled: + case FdbError.OperationCancelled: // happens if a future is cancelled (probably a watch) + case FdbError.TransactionCancelled: // happens if a transaction is cancelled (via its own parent CT, or via tr.Cancel()) { return CATEGORY_CANCELLED; } @@ -529,8 +673,8 @@ protected override void Dispose(bool disposing) } /// Start a new async operation - /// Result of the operation - /// Argument passed to the generator + /// Type of the result of the operation + /// Type of the argument passed to the generator /// Lambda called to produce the future handle /// Argument passed to . It will not be used after the handle has been constructed /// Lambda called once the future completes (successfully) @@ -538,15 +682,16 @@ protected override void Dispose(bool disposing) /// Optional cancellation token used to cancel the task from an external source. /// Optional label, used for logging and troubleshooting purpose (by default the name of the caller) /// - protected Task StartNewFuture( - Func generator, + protected Task RunAsync( + [NotNull] Func generator, TArg argument, - Func selector, + [NotNull] Func selector, object state, CancellationToken ct, [CallerMemberName] string label = null ) { + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); bool mustDispose = true; IntPtr h = IntPtr.Zero; @@ -568,7 +713,7 @@ protected Task StartNewFuture( } } - protected Task StartNewFutures( + protected Task RunAsync( int count, Action generator, TArg arg, diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index f6c729ddd..9ff05330f 100644 --- a/FoundationDB.Client/Subspaces/Fdb.Directory.cs +++ b/FoundationDB.Client/Subspaces/Fdb.Directory.cs @@ -75,7 +75,7 @@ public static async Task OpenNamedPartitionAsync(string clusterFil FdbSubspace rootSpace = FdbSubspace.Empty; try { - db = await Fdb.OpenInternalAsync(clusterFile, dbName, rootSpace, readOnly: false, cancellationToken: cancellationToken).ConfigureAwait(false); + db = (FdbDatabase) (await Fdb.OpenInternalAsync(clusterFile, dbName, rootSpace, readOnly: false, cancellationToken: cancellationToken).ConfigureAwait(false)); var rootLayer = FdbDirectoryLayer.Create(rootSpace); if (Logging.On) Logging.Verbose(typeof(Fdb.Directory), "OpenNamedPartitionAsync", String.Format("Opened root layer of database {0} using cluster file '{1}'", db.Name, db.Cluster.Path)); diff --git a/FoundationDB.Tests/FdbTest.cs b/FoundationDB.Tests/FdbTest.cs index 97d47900c..70443c76e 100644 --- a/FoundationDB.Tests/FdbTest.cs +++ b/FoundationDB.Tests/FdbTest.cs @@ -142,24 +142,24 @@ protected async Task DeleteSubspace(IFdbDatabase db, FdbSubspace subspace) // These methods are just there to help with the problem of culture-aware string formatting - protected void Log(string text) + protected static void Log(string text) { Console.WriteLine(text); } - protected void Log(string format, object arg0) + protected static void Log(string format, object arg0) { - Console.WriteLine(String.Format(CultureInfo.InvariantCulture, format, arg0)); + Log(String.Format(CultureInfo.InvariantCulture, format, arg0)); } - protected void Log(string format, object arg0, object arg1) + protected static void Log(string format, object arg0, object arg1) { - Console.WriteLine(String.Format(CultureInfo.InvariantCulture, format, arg0, arg1)); + Log(String.Format(CultureInfo.InvariantCulture, format, arg0, arg1)); } - protected void Log(string format, params object[] args) + protected static void Log(string format, params object[] args) { - Console.WriteLine(String.Format(CultureInfo.InvariantCulture, format, args)); + Log(String.Format(CultureInfo.InvariantCulture, format, args)); } #endregion diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index a8950e5f3..3eb1bc863 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -270,11 +270,7 @@ public async Task Test_Cancelling_Transaction_During_Commit_Should_Abort_Task() Assume.That(t.IsCompleted, Is.False, "Commit task already completed before having a chance to cancel"); tr.Cancel(); - await TestHelpers.AssertThrowsFdbErrorAsync( - () => t, - FdbError.TransactionCancelled, - "Cancelling a transaction that is writing to the server should fail the commit task" - ); + Assert.Throws(async () => await t, "Cancelling a transaction that is writing to the server should fail the commit task"); } } } @@ -1902,7 +1898,7 @@ public async Task Test_Simple_Read_Transaction() tr.Set(b, Slice.FromString("BAZ")); tr.Set(c, Slice.FromString("BAT")); tr.ClearRange(a, c); - + //tr.ClearRange(location.Concat(Slice.FromString("A")), location.Concat(Slice.FromString("Z"))); //tr.Set(location.Concat(Slice.FromString("C")), Slice.Empty); @@ -1927,125 +1923,135 @@ public async Task Test_Simple_Read_Transaction() [Test, Category("LongRunning")] public async Task Test_BadPractice_Future_Fuzzer() { - const int DURATION_SEC = 30; + const int DURATION_SEC = 10; const int R = 100; - using (var db = await OpenTestDatabaseAsync()) + try { - var location = db.Partition("Fuzzer"); + using (var db = await OpenTestDatabaseAsync()) + { + Console.WriteLine("B"); + var location = db.Partition("Fuzzer"); - var rnd = new Random(); - int seed = rnd.Next(); - Log("Using random seeed {0}", seed); - rnd = new Random(seed); + var rnd = new Random(); + int seed = rnd.Next(); + Log("Using random seeed {0}", seed); + rnd = new Random(seed); - await db.WriteAsync((tr) => - { - for (int i = 0; i < R; i++) + await db.WriteAsync((tr) => { - tr.Set(location.Pack(i), Slice.FromInt32(i)); - } - }, this.Cancellation); + for (int i = 0; i < R; i++) + { + tr.Set(location.Pack(i), Slice.FromInt32(i)); + } + }, this.Cancellation); - var start = DateTime.UtcNow; - Log("This test will run for {0} seconds", DURATION_SEC); + Console.WriteLine("C"); - int time = 0; + var start = DateTime.UtcNow; + Log("This test will run for {0} seconds", DURATION_SEC); - List m_alive = new List(); - while (DateTime.UtcNow - start < TimeSpan.FromSeconds(DURATION_SEC)) - { - switch (rnd.Next(10)) + int time = 0; + + var line = new StringBuilder(256); + + var alive = new List(100); + var lastCheck = start; + while (DateTime.UtcNow - start < TimeSpan.FromSeconds(DURATION_SEC)) { - case 0: + int x = rnd.Next(10); + + if (x == 0) { // start a new transaction - Console.Write('T'); + line.Append('T'); var tr = db.BeginTransaction(FdbTransactionMode.Default, this.Cancellation); - m_alive.Add(tr); - break; + alive.Add(tr); } - case 1: + else if (x == 1) { // drop a random transaction - if (m_alive.Count == 0) continue; - Console.Write('L'); - int p = rnd.Next(m_alive.Count); + if (alive.Count == 0) continue; + line.Append('L'); + int p = rnd.Next(alive.Count); - m_alive.RemoveAt(p); - //no dispose - break; + alive.RemoveAt(p); + //no dispose! } - case 2: + else if (x == 2) { // dispose a random transaction - if (m_alive.Count == 0) continue; - Console.Write('D'); - int p = rnd.Next(m_alive.Count); + if (alive.Count == 0) continue; + line.Append('D'); + int p = rnd.Next(alive.Count); - var tr = m_alive[p]; + var tr = alive[p]; + alive.RemoveAt(p); tr.Dispose(); - m_alive.RemoveAt(p); - break; } - case 3: - { // GC! - Console.Write('C'); + else if (x == 3) + { // get read version + line.Append('R'); var tr = db.BeginTransaction(FdbTransactionMode.ReadOnly, this.Cancellation); - m_alive.Add(tr); + alive.Add(tr); await tr.GetReadVersionAsync(); - break; } - - case 4: - case 5: - case 6: - { // read a random value from a random transaction - Console.Write('G'); - if (m_alive.Count == 0) break; - int p = rnd.Next(m_alive.Count); - var tr = m_alive[p]; - - int x = rnd.Next(R); - try - { - var res = await tr.GetAsync(location.Pack(x)); + else + { + if (x % 2 == 0) + { // read a random value from a random transaction + if (alive.Count == 0) continue; + line.Append('G'); + int p = rnd.Next(alive.Count); + var tr = alive[p]; + + int k = rnd.Next(R); + try + { + await tr.GetAsync(location.Pack(k)); + } + catch (FdbException) + { + line.Append('!'); + alive.RemoveAt(p); + tr.Dispose(); + } } - catch (FdbException) - { - Console.Write('!'); + else + { // read a random value, but drop the task + if (alive.Count == 0) continue; + line.Append('g'); + int p = rnd.Next(alive.Count); + var tr = alive[p]; + + int k = rnd.Next(R); + var t = tr.GetAsync(location.Pack(k)).ContinueWith((_) => { var err = _.Exception; }, TaskContinuationOptions.OnlyOnFaulted); + // => t is not stored } - break; } - case 7: - case 8: - case 9: - { // read a random value, but drop the task - Console.Write('g'); - if (m_alive.Count == 0) break; - int p = rnd.Next(m_alive.Count); - var tr = m_alive[p]; - - int x = rnd.Next(R); - var t = tr.GetAsync(location.Pack(x)).ContinueWith((_) => Console.Write('!'), TaskContinuationOptions.NotOnRanToCompletion); - // => t is not stored - break; + + if ((++time) % 10 == 0 && DateTime.UtcNow - lastCheck >= TimeSpan.FromSeconds(1)) + { + Log(line.ToString()); + line.Clear(); + Log("State: {0}", alive.Count); + //Log("Performing full GC"); + //GC.Collect(2); + //GC.WaitForPendingFinalizers(); + //GC.Collect(2); + lastCheck = DateTime.UtcNow; } - } - if ((time++) % 80 == 0) - { - Console.WriteLine(); - Log("State: {0}", m_alive.Count); - Console.Write('C'); - GC.Collect(); - GC.WaitForPendingFinalizers(); - GC.Collect(); + await Task.Delay(1); + } + GC.Collect(); + GC.WaitForPendingFinalizers(); + GC.Collect(); } - - GC.Collect(); - GC.WaitForPendingFinalizers(); - GC.Collect(); + } + finally + { + Log("Test methods completed!"); } } diff --git a/FoundationDb.Client.sln.DotSettings b/FoundationDb.Client.sln.DotSettings index 47110b55c..d76791c18 100644 --- a/FoundationDb.Client.sln.DotSettings +++ b/FoundationDb.Client.sln.DotSettings @@ -65,4 +65,6 @@ <Policy Inspect="True" Prefix="s_" Suffix="" Style="aaBb"><ExtraRule Prefix="" Suffix="" Style="AaBb" /></Policy> <Policy Inspect="True" Prefix="" Suffix="" Style="AaBb"><ExtraRule Prefix="m_" Suffix="" Style="aaBb" /></Policy> True - True \ No newline at end of file + True + True + False \ No newline at end of file From 79552dfba70ba2ef5a14173641004f0332c63099 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 16 Feb 2015 10:10:55 +0100 Subject: [PATCH 07/13] Added the CancellationToken.RegisterWithoutEC() hack to not pay the cost of copying the execution context for nothing - Used internally to trigger cancellation, and we do not care about the ExecutionContext to do that. --- .../Async/AsyncCancellableMutex.cs | 6 ++-- FoundationDB.Client/Async/TaskHelpers.cs | 30 +++++++++++++++++++ FoundationDB.Client/FdbTransaction.cs | 2 +- .../Native/Futures/FdbFutureContext.cs | 5 ++-- 4 files changed, 36 insertions(+), 7 deletions(-) diff --git a/FoundationDB.Client/Async/AsyncCancellableMutex.cs b/FoundationDB.Client/Async/AsyncCancellableMutex.cs index 2e4a16990..d73868a14 100644 --- a/FoundationDB.Client/Async/AsyncCancellableMutex.cs +++ b/FoundationDB.Client/Async/AsyncCancellableMutex.cs @@ -83,7 +83,7 @@ public AsyncCancelableMutex(CancellationToken ct) { if (ct.CanBeCanceled) { - m_ctr = ct.Register(s_cancellationCallback, new WeakReference(this), useSynchronizationContext: false); + m_ctr = ct.RegisterWithoutEC(s_cancellationCallback, new WeakReference(this)); } GC.SuppressFinalize(this); } @@ -130,12 +130,12 @@ public bool Abort(bool async = false) private static void SetDefered(AsyncCancelableMutex mutex) { - ThreadPool.QueueUserWorkItem((state) => ((AsyncCancelableMutex)state).TrySetResult(null), mutex); + ThreadPool.UnsafeQueueUserWorkItem((state) => ((AsyncCancelableMutex)state).TrySetResult(null), mutex); } private static void CancelDefered(AsyncCancelableMutex mutex) { - ThreadPool.QueueUserWorkItem((state) => ((AsyncCancelableMutex)state).TrySetCanceled(), mutex); + ThreadPool.UnsafeQueueUserWorkItem((state) => ((AsyncCancelableMutex)state).TrySetCanceled(), mutex); } } diff --git a/FoundationDB.Client/Async/TaskHelpers.cs b/FoundationDB.Client/Async/TaskHelpers.cs index 9334e5e1b..9fdc4635d 100644 --- a/FoundationDB.Client/Async/TaskHelpers.cs +++ b/FoundationDB.Client/Async/TaskHelpers.cs @@ -400,6 +400,36 @@ public static void Observe(Task task) } } + private delegate CancellationTokenRegistration RegisterWithoutECDelegate(ref CancellationToken ct, Action callback, object state); + private static readonly RegisterWithoutECDelegate RegisterWithoutECHandler = GetRegisterWithoutECDelegate(); + + [NotNull] + private static RegisterWithoutECDelegate GetRegisterWithoutECDelegate() + { + try + { + // CancellationToken.Register(..., useExecutionContext) is "private", and all the public version of Register pass true, which does costly allocations (capturing context, ...) + // There is however CancellationToken.InternalRegisterWithoutEC which is internal and pass false. + // => we will attempt to create a delegate to call the internal method - if possible - or fallback to the default version of Register, if this is not possible. + var method = typeof(CancellationToken).GetMethod("InternalRegisterWithoutEC", System.Reflection.BindingFlags.Instance | System.Reflection.BindingFlags.NonPublic, null, new[] { typeof(Action), typeof(object) }, null); + if (method != null) + { + return (RegisterWithoutECDelegate)Delegate.CreateDelegate(typeof(RegisterWithoutECDelegate), null, method); + } + } + catch + { } + + return (ref CancellationToken token, Action callback, object state) => token.Register(callback, state); + } + + /// Version of CancellationToken.Register() that does not propagate the current ExecutionContext to the callback (faster, but unsafe!) + /// This should only be used with callbacks that do not execute user-provided code! + internal static CancellationTokenRegistration RegisterWithoutEC(this CancellationToken ct, [NotNull] Action callback, object state) + { + return RegisterWithoutECHandler(ref ct, callback, state); + } + /// Safely cancel a CancellationTokenSource /// CancellationTokenSource that needs to be cancelled public static void SafeCancel(this CancellationTokenSource source) diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index fbeac558b..5722a75d8 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -118,7 +118,7 @@ internal FdbTransaction(FdbDatabase db, FdbOperationContext context, int id, IFd } else { - m_ctr = m_cancellation.Register(CancellationHandler, this); + m_ctr = m_cancellation.RegisterWithoutEC(CancellationHandler, this); } } diff --git a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs index f3e5eda2b..94edc38fc 100644 --- a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs +++ b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs @@ -567,10 +567,9 @@ internal static CancellationTokenRegistration RegisterForCancellation(IFdbFuture { //note: if the token is already cancelled, the callback handler will run inline and any exception would bubble up here //=> this is not a problem because the ctor already has a try/catch that will clean up everything - return cancellationToken.Register( + return cancellationToken.RegisterWithoutEC( (_state) => { CancellationHandler(_state); }, - future, - false + future ); } From 990b053a847ebeb1221a48e713f8e9d83b86f91e Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 16 Feb 2015 19:30:40 +0100 Subject: [PATCH 08/13] No custom CancellationToken for regular read operations on Snapshotted transaction - Cancellation will be handled by tr.Cancel() internally --- FoundationDB.Client/FdbTransaction.Snapshot.cs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/FoundationDB.Client/FdbTransaction.Snapshot.cs b/FoundationDB.Client/FdbTransaction.Snapshot.cs index 9c1e7bce7..b8171d51e 100644 --- a/FoundationDB.Client/FdbTransaction.Snapshot.cs +++ b/FoundationDB.Client/FdbTransaction.Snapshot.cs @@ -119,7 +119,7 @@ public Task GetAsync(Slice key) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAsync", String.Format("Getting value for '{0}'", key.ToString())); #endif - return m_parent.m_handler.GetAsync(key, snapshot: true, cancellationToken: m_parent.m_cancellation); + return m_parent.m_handler.GetAsync(key, snapshot: true, cancellationToken: CancellationToken.None); } public Task GetValuesAsync(Slice[] keys) @@ -134,7 +134,7 @@ public Task GetValuesAsync(Slice[] keys) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetValuesAsync", String.Format("Getting batch of {0} values ...", keys.Length)); #endif - return m_parent.m_handler.GetValuesAsync(keys, snapshot: true, cancellationToken: m_parent.m_cancellation); + return m_parent.m_handler.GetValuesAsync(keys, snapshot: true, cancellationToken: CancellationToken.None); } public async Task GetKeyAsync(FdbKeySelector selector) @@ -147,7 +147,7 @@ public async Task GetKeyAsync(FdbKeySelector selector) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeyAsync", String.Format("Getting key '{0}'", selector.ToString())); #endif - var key = await m_parent.m_handler.GetKeyAsync(selector, snapshot: true, cancellationToken: m_parent.m_cancellation).ConfigureAwait(false); + var key = await m_parent.m_handler.GetKeyAsync(selector, snapshot: true, cancellationToken: CancellationToken.None).ConfigureAwait(false); // don't forget to truncate keys that would fall outside of the database's globalspace ! return m_parent.m_database.BoundCheck(key); @@ -167,7 +167,7 @@ public Task GetKeysAsync(FdbKeySelector[] selectors) if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeysCoreAsync", String.Format("Getting batch of {0} keys ...", selectors.Length)); #endif - return m_parent.m_handler.GetKeysAsync(selectors, snapshot: true, cancellationToken: m_parent.m_cancellation); + return m_parent.m_handler.GetKeysAsync(selectors, snapshot: true, cancellationToken: CancellationToken.None); } public Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options, int iteration) @@ -183,7 +183,7 @@ public Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySe // The iteration value is only needed when in iterator mode, but then it should start from 1 if (iteration == 0) iteration = 1; - return m_parent.m_handler.GetRangeAsync(beginInclusive, endExclusive, options, iteration, snapshot: true, cancellationToken: m_parent.m_cancellation); + return m_parent.m_handler.GetRangeAsync(beginInclusive, endExclusive, options, iteration, snapshot: true, cancellationToken: CancellationToken.None); } public FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options) @@ -196,7 +196,7 @@ public FdbRangeQuery> GetRange(FdbKeySelector beginIn public Task GetAddressesForKeyAsync(Slice key) { EnsureCanRead(); - return m_parent.m_handler.GetAddressesForKeyAsync(key, cancellationToken: m_parent.m_cancellation); + return m_parent.m_handler.GetAddressesForKeyAsync(key, CancellationToken.None); } void IFdbReadOnlyTransaction.Cancel() From e845b03ae86791d45a721cd235503699f2a4a784 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 16 Feb 2015 19:32:04 +0100 Subject: [PATCH 09/13] FutureContext: make sure to remove the future from the context when it is done, and more locking --- .../Native/Futures/FdbFuture.cs | 4 +- .../Native/Futures/FdbFutureContext.cs | 58 +++++++++++-------- 2 files changed, 35 insertions(+), 27 deletions(-) diff --git a/FoundationDB.Client/Native/Futures/FdbFuture.cs b/FoundationDB.Client/Native/Futures/FdbFuture.cs index a72053778..4e944ae9c 100644 --- a/FoundationDB.Client/Native/Futures/FdbFuture.cs +++ b/FoundationDB.Client/Native/Futures/FdbFuture.cs @@ -94,12 +94,12 @@ public void Cancel() protected abstract void OnCancel(); - protected void PublishResult(T result) + internal void PublishResult(T result) { TrySetResult(result); } - protected void PublishError(Exception error, FdbError code) + internal void PublishError(Exception error, FdbError code) { if (error != null) { diff --git a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs index 94edc38fc..30b8853a0 100644 --- a/FoundationDB.Client/Native/Futures/FdbFutureContext.cs +++ b/FoundationDB.Client/Native/Futures/FdbFutureContext.cs @@ -296,6 +296,11 @@ private bool OnFutureReady(IntPtr handle, IntPtr cookie, bool fromNetworkThread, if (future.Visit(handle)) { // future is ready to process all the results + lock (m_futures) + { + m_futures.Remove(futureId); + } + if (fromNetworkThread) { ThreadPool.UnsafeQueueUserWorkItem((state) => ((IFdbFuture)state).OnReady(), future); @@ -371,24 +376,27 @@ string label lock (m_futures) { - //TODO: marke the future as "registered" (must unreg when it fires?) m_futures[futureId] = future; - } - var err = FdbNative.FutureSetCallback(handle, GlobalCallback, cookie); - if (!Fdb.Success(err)) - { // the callback will not fire, so we have to abort the future immediately - future.TrySetException(Fdb.MapToException(err)); - return future.Task; + // note: if the future just got ready, the callback will fire inline (as of v3.0) + // => if this happens, the callback defer the execution to the ThreadPool and returns immediately + var err = FdbNative.FutureSetCallback(handle, GlobalCallback, cookie); + if (!Fdb.Success(err)) + { // the callback will not fire, so we have to abort the future immediately + future.PublishError(null, err); + } + else + { + mustDispose = false; + } } - mustDispose = false; return future.Task; } catch (Exception e) { if (future != null) { - future.TrySetException(e); + future.PublishError(e, FdbError.UnknownError); return future.Task; } throw; @@ -449,19 +457,15 @@ string label } if (ready) { +#if DEBUG_FUTURES Debug.WriteLine("FutureArray.{0} [{1}] already completed!", label, tmp.Length); +#endif cookie = IntPtr.Zero; mustDispose = false; future.OnReady(); return future.Task; } - lock (m_futures) - { - //TODO: mark the future as "registered" (must unreg when it fires?) - m_futures[futureId] = future; - } - if (ct.CanBeCanceled) { future.m_ctr = RegisterForCancellation(future, ct); @@ -472,20 +476,24 @@ string label } } - for (int i = 0; i < handles.Length; i++) + lock (m_futures) { - var err = FdbNative.FutureSetCallback(handles[i], GlobalCallback, cookie); - if (Fdb.Success(err)) - { - handles[i] = IntPtr.Zero; - } - else + m_futures[futureId] = future; + + // since the callbacks can fire inline, we have to make sure that we finish setting everything up under the lock + for (int i = 0; i < handles.Length; i++) { - // mute this future + FdbError err = FdbNative.FutureSetCallback(handles[i], GlobalCallback, cookie); + if (Fdb.Success(err)) + { + handles[i] = IntPtr.Zero; + continue; + } + + // we have to cleanup everything, and mute this future lock (m_futures) { m_futures.Remove(futureId); - //TODO: mark the future as "unregistered" for (int j = i + 1; j < handles.Length; j++) { tmp[j] = IntPtr.Zero; @@ -502,7 +510,7 @@ string label { if (future != null) { - future.TrySetException(e); + future.PublishError(e, FdbError.UnknownError); return future.Task; } throw; From fe67fa54f2cc0b00c8ba69ec7959f8adfb682e2a Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Feb 2015 16:20:11 +0100 Subject: [PATCH 10/13] Slice: simplified SliceHelper to always call memmove and memcmp - Using a profiler, and looking at the x64 assembly generated by the JIT, it seems that always calling memcmp / memmove is fast enough - Added #define MEASURE to count and measure the invocation of memory copy and compares to get a feel of the most used sizes by typical applications - Count 0, 1 and 2, which are most impacted by calling memcmp or memcpy, are less frequent. Most frequent in a reference app where size 5 (corresponds to a 4 bytes integer + the type prefix in tuple encoding) or 17 bytes (GUIDs in tuples). - Reduced the number of method calls needed to invoke memcmp and memmove - Made EnsureSliceIsValid and EnsureBufferIsValid inlined by default, to remove one more method call --- FoundationDB.Client/Utils/Slice.cs | 4 +- FoundationDB.Client/Utils/SliceComparer.cs | 5 +- FoundationDB.Client/Utils/SliceHelpers.cs | 506 ++++----------------- FoundationDB.Tests/Layers/TupleFacts.cs | 24 +- FoundationDB.Tests/Utils/SliceFacts.cs | 23 +- 5 files changed, 133 insertions(+), 429 deletions(-) diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs index 0a421e358..9b910dfd2 100644 --- a/FoundationDB.Client/Utils/Slice.cs +++ b/FoundationDB.Client/Utils/Slice.cs @@ -191,7 +191,7 @@ public static Slice Repeat(byte value, int count) if (count == 0) return Slice.Empty; var res = new byte[count]; - SliceHelpers.SetBytes(res, 0, count, value); + if (value != 0) SliceHelpers.SetBytes(res, 0, count, value); return new Slice(res, 0, res.Length); } @@ -1696,7 +1696,7 @@ public void WriteTo(byte[] buffer, ref int cursor) if (this.Count > 0) { - SliceHelpers.CopyBytes(buffer, cursor, this.Array, this.Offset, this.Count); + SliceHelpers.CopyBytesUnsafe(buffer, cursor, this.Array, this.Offset, this.Count); cursor += this.Count; } } diff --git a/FoundationDB.Client/Utils/SliceComparer.cs b/FoundationDB.Client/Utils/SliceComparer.cs index 83d911b51..e7919f09b 100644 --- a/FoundationDB.Client/Utils/SliceComparer.cs +++ b/FoundationDB.Client/Utils/SliceComparer.cs @@ -50,10 +50,13 @@ private SliceComparer() /// public int Compare(Slice x, Slice y) { + SliceHelpers.EnsureSliceIsValid(ref x); + SliceHelpers.EnsureSliceIsValid(ref y); + //REVIEW: cmp(Nil, Empty) returns 0 but Nil != Empty ? if (x.Count == 0) return y.Count == 0 ? 0 : -1; if (y.Count == 0) return +1; - return SliceHelpers.CompareBytes(x.Array, x.Offset, x.Count, y.Array, y.Offset, y.Count); + return SliceHelpers.CompareBytesUnsafe(x.Array, x.Offset, x.Count, y.Array, y.Offset, y.Count); } /// Checks if two slices are equal. diff --git a/FoundationDB.Client/Utils/SliceHelpers.cs b/FoundationDB.Client/Utils/SliceHelpers.cs index c4d93a7fb..0f2eaae8c 100644 --- a/FoundationDB.Client/Utils/SliceHelpers.cs +++ b/FoundationDB.Client/Utils/SliceHelpers.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define USE_NATIVE_MEMORY_OPERATORS +//#define MEASURE namespace FoundationDB.Client { @@ -34,15 +34,20 @@ namespace FoundationDB.Client using JetBrains.Annotations; using System; using System.Runtime.CompilerServices; - using System.Runtime.ConstrainedExecution; using System.Runtime.InteropServices; using System.Security; internal static class SliceHelpers { +#if !NET_4_0 + [MethodImpl(MethodImplOptions.AggressiveInlining)] +#endif public static void EnsureSliceIsValid(ref Slice slice) { + // this method is used everywhere, and is consistently the top 1 method by callcount when using a profiler, + // so we must make sure that it gets inline whenever possible. + if (slice.Count == 0 && slice.Offset >= 0) return; if (slice.Count < 0 || slice.Offset < 0 || slice.Array == null || slice.Offset + slice.Count > slice.Array.Length) { @@ -53,7 +58,7 @@ public static void EnsureSliceIsValid(ref Slice slice) /// Reject an invalid slice by throw an error with the appropriate diagnostic message. /// Slice that is being naugthy [ContractAnnotation("=> halt")] - public static void ThrowMalformedSlice(Slice slice) + private static void ThrowMalformedSlice(Slice slice) { #if DEBUG // If you break here, that means that a slice is invalid (negative count, offset, ...), which may be a sign of memory corruption! @@ -72,6 +77,9 @@ public static void ThrowMalformedSlice(Slice slice) throw new FormatException("The specified slice is invalid."); } +#if !NET_4_0 + [MethodImpl(MethodImplOptions.AggressiveInlining)] +#endif public static void EnsureBufferIsValid(byte[] array, int offset, int count) { if (count == 0 && offset >= 0) return; @@ -83,7 +91,7 @@ public static void EnsureBufferIsValid(byte[] array, int offset, int count) /// Reject an invalid slice by throw an error with the appropriate diagnostic message. [ContractAnnotation("=> halt")] - public static void ThrowMalformedBuffer(byte[] array, int offset, int count) + private static void ThrowMalformedBuffer(byte[] array, int offset, int count) { if (offset < 0) throw new ArgumentException("The specified segment has a negative offset, which is not legal. This may be a side effect of memory corruption.", "offset"); if (count < 0) throw new ArgumentException("The specified segment has a negative size, which is not legal. This may be a side effect of memory corruption.", "count"); @@ -176,6 +184,11 @@ public static int ComputeHashCodeUnsafe([NotNull] byte[] bytes, int offset, int // } +#if MEASURE + public static int[] CompareHistogram = new int[65536]; + public static double[] CompareDurations = new double[65536]; +#endif + /// Compare two byte segments for equality /// Left buffer /// Start offset in left buffer @@ -202,64 +215,25 @@ public static bool SameBytes(byte[] left, int leftOffset, byte[] right, int righ public static bool SameBytesUnsafe([NotNull] byte[] left, int leftOffset, [NotNull] byte[] right, int rightOffset, int count) { Contract.Requires(left != null && leftOffset >= 0 && right != null && rightOffset >= 0 && count >= 0); - - // for very small keys, the cost of pinning and marshalling may be too high - if (count <= 8) - { - while (count-- > 0) - { - if (left[leftOffset++] != right[rightOffset++]) return false; - } - return true; - } - - if (object.ReferenceEquals(left, right)) - { // In cases where the keys are backed by the same buffer, we don't need to pin the same buffer twice - - if (leftOffset == rightOffset) - { // same segment in the same buffer - return true; - } - - unsafe - { - fixed (byte* ptr = left) - { - return 0 == CompareMemoryUnsafe(ptr + leftOffset, ptr + rightOffset, count); - } - } - } - else +#if MEASURE + int n = count; + if (n < SliceHelpers.CompareHistogram.Length) ++SliceHelpers.CompareHistogram[n]; + var sw = System.Diagnostics.Stopwatch.StartNew(); +#endif + int c; + unsafe { - unsafe + fixed (byte* pLeft = &left[leftOffset]) + fixed (byte* pRight = &right[rightOffset]) { - fixed (byte* pLeft = left) - fixed (byte* pRight = right) - { - return 0 == CompareMemoryUnsafe(pLeft + leftOffset, pRight + rightOffset, count); - } + c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); } } - } - - /// Compare two byte segments lexicographically - /// Left buffer - /// Start offset in left buffer - /// Number of bytes in left buffer - /// Right buffer - /// Start offset in right buffer - /// Number of bytes in right buffer - /// Returns zero if segments are identical (same bytes), a negative value if left is lexicographically less than right, or a positive value if left is lexicographically greater than right - /// The comparison algorithm respect the following: - /// * "A" < "B" - /// * "A" < "AA" - /// * "AA" < "B" - public static int CompareBytes(byte[] left, int leftOffset, int leftCount, byte[] right, int rightOffset, int rightCount) - { - SliceHelpers.EnsureBufferIsValid(left, leftOffset, leftCount); - SliceHelpers.EnsureBufferIsValid(right, rightOffset, rightCount); - - return CompareBytesUnsafe(left, leftOffset, leftCount, right, rightOffset, rightCount); +#if MEASURE + sw.Stop(); + if (n < SliceHelpers.CompareDurations.Length) SliceHelpers.CompareDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); +#endif + return c == 0; } /// Compare two byte segments lexicographically, without validating the arguments @@ -278,51 +252,32 @@ public static int CompareBytesUnsafe([NotNull] byte[] left, int leftOffset, int { Contract.Requires(left != null && right != null && leftOffset >= 0 && leftCount >= 0 && rightOffset >= 0 && rightCount >= 0); - if (object.ReferenceEquals(left, right)) - { // In cases where the keys are backed by the same buffer, we don't need to pin the same buffer twice - - if (leftCount == rightCount && leftOffset == rightOffset) - { // same segment in the same buffer - return 0; - } - - unsafe - { - fixed (byte* ptr = left) - { - int n = CompareMemoryUnsafe(ptr + leftOffset, ptr + rightOffset, Math.Min(leftCount, rightCount)); - return n != 0 ? n : leftCount - rightCount; - } - } - } - else + int count = Math.Min(leftCount, rightCount); +#if MEASURE + int n = count; + if (n < SliceHelpers.CompareHistogram.Length) ++SliceHelpers.CompareHistogram[n]; + var sw = System.Diagnostics.Stopwatch.StartNew(); +#endif + int c; + unsafe { - unsafe + fixed (byte* pLeft = &left[leftOffset]) + fixed (byte* pRight = &right[rightOffset]) { - fixed (byte* pLeft = left) - fixed (byte* pRight = right) - { - int n = CompareMemoryUnsafe(pLeft + leftOffset, pRight + rightOffset, Math.Min(leftCount, rightCount)); - return n != 0 ? n : leftCount - rightCount; - } + c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); } } +#if MEASURE + sw.Stop(); + if (n < SliceHelpers.CompareDurations.Length) SliceHelpers.CompareDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); +#endif + return c != 0 ? c : leftCount - rightCount; } - /// Copy the content of a byte segment into another. CAUTION: The arguments are NOT in the same order as Buffer.BlockCopy() or Array.Copy() ! - /// Destination buffer - /// Offset in destination buffer - /// Source buffer - /// Offset in source buffer - /// Number of bytes to copy - /// CAUTION: THE ARGUMENTS ARE REVERSED! They are in the same order as memcpy() and memmove(), with destination first, and source second! - public static void CopyBytes(byte[] dst, int dstOffset, byte[] src, int srcOffset, int count) - { - SliceHelpers.EnsureBufferIsValid(dst, dstOffset, count); - SliceHelpers.EnsureBufferIsValid(src, srcOffset, count); - - CopyBytesUnsafe(dst, dstOffset, src, srcOffset, count); - } +#if MEASURE + public static int[] CopyHistogram = new int[65536]; + public static double[] CopyDurations = new double[65536]; +#endif /// Copy the content of a byte segment into another, without validating the arguments. CAUTION: The arguments are NOT in the same order as Buffer.BlockCopy() or Array.Copy() ! /// Destination buffer @@ -331,40 +286,26 @@ public static void CopyBytes(byte[] dst, int dstOffset, byte[] src, int srcOffse /// Offset in source buffer /// Number of bytes to copy /// CAUTION: THE ARGUMENTS ARE REVERSED! They are in the same order as memcpy() and memmove(), with destination first, and source second! - public static void CopyBytesUnsafe([NotNull] byte[] dst, int dstOffset, [NotNull] byte[] src, int srcOffset, int count) + public static unsafe void CopyBytesUnsafe([NotNull] byte[] dst, int dstOffset, [NotNull] byte[] src, int srcOffset, int count) { Contract.Requires(dst != null && src != null && dstOffset >= 0 && srcOffset >= 0 && count >= 0); - if (count <= 8) - { // for very small keys, the cost of pinning and marshalling may be to high - - while (count-- > 0) - { - dst[dstOffset++] = src[srcOffset++]; - } - } - else if (object.ReferenceEquals(dst, src)) - { // In cases where the keys are backed by the same buffer, we don't need to pin the same buffer twice +#if MEASURE + int n = count; + if (n < SliceHelpers.CopyHistogram.Length) ++SliceHelpers.CopyHistogram[n]; + var sw = System.Diagnostics.Stopwatch.StartNew(); +#endif - unsafe - { - fixed (byte* ptr = dst) - { - MoveMemoryUnsafe(ptr + dstOffset, ptr + srcOffset, count); - } - } - } - else + fixed (byte* pDst = &dst[dstOffset]) + fixed (byte* pSrc = &src[srcOffset]) { - unsafe - { - fixed (byte* pDst = dst) - fixed (byte* pSrc = src) - { - MoveMemoryUnsafe(pDst + dstOffset, pSrc + srcOffset, count); - } - } + NativeMethods.memmove(pDst, pSrc, new IntPtr(count)); } + +#if MEASURE + sw.Stop(); + if (n < SliceHelpers.CopyDurations.Length) SliceHelpers.CopyDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); +#endif } /// Copy the content of a native byte segment into a managed segment, without validating the arguments. @@ -377,27 +318,21 @@ public static unsafe void CopyBytesUnsafe([NotNull] byte[] dst, int dstOffset, b { Contract.Requires(dst != null && src != null && dstOffset >= 0 && count >= 0); - if (count <= 8) - { - while (count-- > 0) - { - dst[dstOffset++] = *src++; - } - } - else +#if MEASURE + int n = count; + if (n < SliceHelpers.CopyHistogram.Length) ++SliceHelpers.CopyHistogram[n]; + var sw = System.Diagnostics.Stopwatch.StartNew(); +#endif + + fixed (byte* pDst = &dst[dstOffset]) { - fixed (byte* ptr = dst) - { - MoveMemoryUnsafe(ptr + dstOffset, src, count); - } + NativeMethods.memmove(pDst, src, new IntPtr(count)); } - } - /// Fill the content of a managed buffer with the same byte repeated - public static void SetBytes(byte[] bytes, byte value) - { - if (bytes == null) throw new ArgumentNullException("bytes"); - SetBytes(bytes, 0, bytes.Length, value); +#if MEASURE + sw.Stop(); + if (n < SliceHelpers.CopyDurations.Length) SliceHelpers.CopyDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); +#endif } /// Fill the content of a managed segment with the same byte repeated @@ -405,22 +340,11 @@ public static void SetBytes(byte[] bytes, int offset, int count, byte value) { SliceHelpers.EnsureBufferIsValid(bytes, offset, count); - if (count <= 8) - { // for very small keys, the cost of pinning and marshalling may be to high - - while (count-- > 0) - { - bytes[offset++] = value; - } - } - else + unsafe { - unsafe + fixed (byte* ptr = &bytes[offset]) { - fixed (byte* ptr = bytes) - { - SetMemoryUnsafe(ptr + offset, value, count); - } + NativeMethods.memset(ptr, value, new IntPtr(count)); } } } @@ -431,273 +355,11 @@ public static unsafe void SetBytes(byte* bytes, int count, byte value) if (bytes == null) throw new ArgumentNullException("bytes"); if (count < 0) throw new ArgumentException("Count cannot be a negative number.", "count"); - if (count <= 8) - { // for very small keys, the cost of pinning and marshalling may be to high - - while (count-- > 0) - { - *bytes++ = value; - } - } - else - { - SetMemoryUnsafe(bytes, value, count); - } + NativeMethods.memset(bytes, value, new IntPtr(count)); } - /// Dangerously copy native memory from one location to another - /// Where to copy the bytes - /// Where to read the bytes - /// Number of bytes to copy - [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] -#if USE_NATIVE_MEMORY_OPERATORS && !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - private static unsafe void MoveMemoryUnsafe(byte* dest, byte* src, int count) - { - Contract.Requires(dest != null && src != null && count >= 0); - -#if USE_NATIVE_MEMORY_OPERATORS - NativeMethods.memmove(dest, src, new IntPtr(count)); -#else - if (count >= 16) - { - do - { - *((int*)(dest + 0)) = *((int*)(src + 0)); - *((int*)(dest + 4)) = *((int*)(src + 4)); - *((int*)(dest + 8)) = *((int*)(src + 8)); - *((int*)(dest + 12)) = *((int*)(src + 12)); - dest += 16; - src += 16; - } - while ((count -= 16) >= 16); - } - if (count > 0) - { - if ((count & 8) != 0) - { - *((int*)(dest + 0)) = *((int*)(src + 0)); - *((int*)(dest + 4)) = *((int*)(src + 4)); - dest += 8; - src += 8; - } - if ((count & 4) != 0) - { - *((int*)dest) = *((int*)src); - dest += 4; - src += 4; - } - if ((count & 2) != 0) - { - *((short*)dest) = *((short*)src); - dest += 2; - src += 2; - } - if ((count & 1) != 0) - { - *dest = *src; - } - } -#endif - } - - /// Dangerously fill native memory with a specific byte - /// Where to fill the bytes - /// Byte to set - /// Number of bytes to set - /// If ==0, you should call - [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] -#if USE_NATIVE_MEMORY_OPERATORS && !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - private static unsafe void SetMemoryUnsafe(byte* dest, byte c, int count) - { - Contract.Requires(dest != null && count >= 0); - -#if USE_NATIVE_MEMORY_OPERATORS - NativeMethods.memset(dest, c, new IntPtr(count)); -#else - int fill32 = c; - fill32 = fill32 << 8 | c; - fill32 = fill32 << 8 | c; - fill32 = fill32 << 8 | c; - - if (count >= 16) - { - do - { - *((int*)(dest + 0)) = fill32; - *((int*)(dest + 4)) = fill32; - *((int*)(dest + 8)) = fill32; - *((int*)(dest + 12)) = fill32; - dest += 16; - } - while ((count -= 16) >= 16); - } - if (count > 0) - { - if ((count & 8) != 0) - { - *((int*)(dest + 0)) = fill32; - *((int*)(dest + 4)) = fill32; - dest += 8; - } - if ((count & 4) != 0) - { - *((int*)dest) = fill32; - dest += 4; - } - if ((count & 2) != 0) - { - *((short*)dest) = (short)fill32; - dest += 2; - } - if ((count & 1) != 0) - { - *dest = c; - } - } -#endif - } - - /// Dangerously clear native memory - /// Where to clear the bytes - /// Number of bytes to clear - [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] -#if USE_NATIVE_MEMORY_OPERATORS && !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - private static unsafe void ClearMemoryUnsafe(byte* dest, int count) - { - Contract.Requires(dest != null && count >= 0); - -#if USE_NATIVE_MEMORY_OPERATORS - NativeMethods.memset(dest, 0, new IntPtr(count)); -#else - if (count >= 16) - { - do - { - *((ulong*)(dest + 0)) = 0UL; - *((ulong*)(dest + 8)) = 0UL; - dest += 16; - } - while ((count -= 16) >= 16); - } - if (count > 0) - { - if ((count & 8) != 0) - { - *((ulong*)(dest)) = 0UL; - dest += 8; - } - if ((count & 4) != 0) - { - *((int*)dest) = 0; - dest += 4; - } - if ((count & 2) != 0) - { - *((short*)dest) = 0; - dest += 2; - } - if ((count & 1) != 0) - { - *dest = 0; - } - } -#endif - } - - /// Returns the offset of the first difference found between two buffers of the same size - /// Pointer to the first byte of the left buffer - /// Pointer to the first byte of the right buffer - /// Number of bytes to compare in both buffers - /// Offset (from the first byte) of the first difference encountered, or -1 if both buffers are identical. - [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] -#if USE_NATIVE_MEMORY_OPERATORS && !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - private static unsafe int CompareMemoryUnsafe(byte* left, byte* right, int count) - { - Contract.Requires(left != null && right != null && count >= 0); - -#if USE_NATIVE_MEMORY_OPERATORS - return NativeMethods.memcmp(left, right, new IntPtr(count)); -#else - - // We want to scan in chunks of 8 bytes, until we find a difference (or there's less than 8 bytes remaining). - // If we find a difference that way, we backtrack and then scan byte per byte to locate the location of the mismatch. - // for the last 1 to 7 bytes, we just do a regular check - - // XOR Comparison: We XOR two 8-bytes chunks together. - // - If all bytes are identical, the XOR result will be 0. - // - If at least one bit is difference, the XOR result will be non-zero, and the first different will be in the first non-zero byte. - - // Identical data: - // left : "11 22 33 44 55 66 77 88" => 0x8877665544332211 - // right: "11 22 33 44 55 66 77 88" => 0x8877665544332211 - // left XOR right => 0x8877665544332211 ^ 0x8877665544332211 = 0 - - // Different data: - // left : "11 22 33 44 55 66 77 88" => 0x8877665544332211 - // right: "11 22 33 44 55 AA BB CC" => 0xCCBBAA5544332211 - // left XOR right =0x8877665544332211 ^ 0xCCBBAA5544332211 = 0x44CCCC0000000000 - // the first non-zero byte is at offset 5 (big-endian) with the value of 0xCC - - byte* start = left; - - //TODO: align the start of the 8-byte scan to an 8-byte aligne memory address ? - - // compares using 8-bytes chunks - while (count >= 8) - { - ulong k = *((ulong*)left) ^ *((ulong*)right); - - if (k != 0) - { // there is difference in these 8 bytes, iterate until we find it - int p = 0; - while ((k & 0xFF) == 0) - { - ++p; - k >>= 8; - } - return left[p] - right[p]; - } - left += 8; - right += 8; - count -= 8; - } - - // if more than 4 bytes remain, check 32 bits at a time - if (count >= 4) - { - if (*((uint*)left) != *((uint*)right)) - { - goto compare_tail; - } - left += 4; - right += 4; - count -= 4; - } - - // from here, there is at mos 3 bytes remaining - - compare_tail: - while (count-- > 0) - { - int n = *(left++) - *(right++); - if (n != 0) return n; - } - return 0; -#endif - } - -#if USE_NATIVE_MEMORY_OPERATORS - [SuppressUnmanagedCodeSecurity] - internal static unsafe class NativeMethods + private static unsafe class NativeMethods { /// Compare characters in two buffers. @@ -727,8 +389,6 @@ internal static unsafe class NativeMethods } -#endif - } } diff --git a/FoundationDB.Tests/Layers/TupleFacts.cs b/FoundationDB.Tests/Layers/TupleFacts.cs index 3a189739a..e989ba43a 100644 --- a/FoundationDB.Tests/Layers/TupleFacts.cs +++ b/FoundationDB.Tests/Layers/TupleFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2015, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without @@ -26,12 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +//#define MEASURE + namespace FoundationDB.Layers.Tuples.Tests { using FoundationDB.Client; using FoundationDB.Client.Converters; using FoundationDB.Client.Tests; - using FoundationDB.Client.Utils; using NUnit.Framework; using System; using System.Collections.Generic; @@ -44,6 +45,25 @@ namespace FoundationDB.Layers.Tuples.Tests public class TupleFacts : FdbTest { +#if MEASURE + [TestFixtureTearDown] + public void DumpStats() + { + Log("# MemCopy:"); + for (int i = 0; i < SliceHelpers.CopyHistogram.Length; i++) + { + if (SliceHelpers.CopyHistogram[i] == 0) continue; + Log("# {0} : {1:N0} ({2:N1} ns, {3:N3} ns/byte)", i, SliceHelpers.CopyHistogram[i], SliceHelpers.CopyDurations[i] / SliceHelpers.CopyHistogram[i], SliceHelpers.CopyDurations[i] / (SliceHelpers.CopyHistogram[i] * i)); + } + Log("# MemCompare:"); + for (int i = 0; i < SliceHelpers.CompareHistogram.Length; i++) + { + if (SliceHelpers.CompareHistogram[i] == 0) continue; + Log("# {0} : {1:N0} ({2:N1} ns, {3:N3} ns/byte)", i, SliceHelpers.CompareHistogram[i], SliceHelpers.CompareDurations[i] / SliceHelpers.CompareHistogram[i], SliceHelpers.CompareDurations[i] / (SliceHelpers.CompareHistogram[i] * i)); + } + } +#endif + #region General Use... [Test] diff --git a/FoundationDB.Tests/Utils/SliceFacts.cs b/FoundationDB.Tests/Utils/SliceFacts.cs index 448554d92..9f23fd444 100644 --- a/FoundationDB.Tests/Utils/SliceFacts.cs +++ b/FoundationDB.Tests/Utils/SliceFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2015, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +//#define MEASURE + namespace FoundationDB.Client.Tests { using FoundationDB.Client; @@ -41,6 +43,25 @@ namespace FoundationDB.Client.Tests public class SliceFacts : FdbTest { +#if MEASURE + [TestFixtureTearDown] + public void DumpStats() + { + Log("# MemCopy:"); + for (int i = 0; i < SliceHelpers.CopyHistogram.Length; i++) + { + if (SliceHelpers.CopyHistogram[i] == 0) continue; + Log("# {0} : {1:N0} ({2:N1} ns, {3:N3} ns/byte)", i, SliceHelpers.CopyHistogram[i], SliceHelpers.CopyDurations[i] / SliceHelpers.CopyHistogram[i], SliceHelpers.CopyDurations[i] / (SliceHelpers.CopyHistogram[i] * i)); + } + Log("# MemCompare:"); + for (int i = 0; i < SliceHelpers.CompareHistogram.Length; i++) + { + if (SliceHelpers.CompareHistogram[i] == 0) continue; + Log("# {0} : {1:N0} ({2:N1} ns, {3:N3} ns/byte)", i, SliceHelpers.CompareHistogram[i], SliceHelpers.CompareDurations[i] / SliceHelpers.CompareHistogram[i], SliceHelpers.CompareDurations[i] / (SliceHelpers.CompareHistogram[i] * i)); + } + } +#endif + [Test] public void Test_Slice_Nil() { From 4b72de6232d9e5dd3e97f1ddfed4883ce4dd12d5 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Feb 2015 16:44:20 +0100 Subject: [PATCH 11/13] Slice: fixed cases where count = 0 - byte* ptr =&buffer[offset] : does a bound check which can fail if the array is empty. - byte* ptr; ptr + offset : does not bound check --- FoundationDB.Client/Utils/SliceHelpers.cs | 59 +++++++++++++++-------- 1 file changed, 40 insertions(+), 19 deletions(-) diff --git a/FoundationDB.Client/Utils/SliceHelpers.cs b/FoundationDB.Client/Utils/SliceHelpers.cs index 0f2eaae8c..6d4446e6b 100644 --- a/FoundationDB.Client/Utils/SliceHelpers.cs +++ b/FoundationDB.Client/Utils/SliceHelpers.cs @@ -215,20 +215,28 @@ public static bool SameBytes(byte[] left, int leftOffset, byte[] right, int righ public static bool SameBytesUnsafe([NotNull] byte[] left, int leftOffset, [NotNull] byte[] right, int rightOffset, int count) { Contract.Requires(left != null && leftOffset >= 0 && right != null && rightOffset >= 0 && count >= 0); + #if MEASURE int n = count; if (n < SliceHelpers.CompareHistogram.Length) ++SliceHelpers.CompareHistogram[n]; var sw = System.Diagnostics.Stopwatch.StartNew(); #endif int c; - unsafe + if (count > 0) { - fixed (byte* pLeft = &left[leftOffset]) - fixed (byte* pRight = &right[rightOffset]) + unsafe { - c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); + fixed (byte* pLeft = &left[leftOffset]) + fixed (byte* pRight = &right[rightOffset]) + { + c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); + } } } + else + { + c = 0; + } #if MEASURE sw.Stop(); if (n < SliceHelpers.CompareDurations.Length) SliceHelpers.CompareDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); @@ -259,12 +267,19 @@ public static int CompareBytesUnsafe([NotNull] byte[] left, int leftOffset, int var sw = System.Diagnostics.Stopwatch.StartNew(); #endif int c; - unsafe + if (count == 0) { - fixed (byte* pLeft = &left[leftOffset]) - fixed (byte* pRight = &right[rightOffset]) + c = 0; + } + else + { + unsafe { - c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); + fixed (byte* pLeft = &left[leftOffset]) + fixed (byte* pRight = &right[rightOffset]) + { + c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); + } } } #if MEASURE @@ -295,13 +310,14 @@ public static unsafe void CopyBytesUnsafe([NotNull] byte[] dst, int dstOffset, [ if (n < SliceHelpers.CopyHistogram.Length) ++SliceHelpers.CopyHistogram[n]; var sw = System.Diagnostics.Stopwatch.StartNew(); #endif - - fixed (byte* pDst = &dst[dstOffset]) - fixed (byte* pSrc = &src[srcOffset]) + if (count > 0) { - NativeMethods.memmove(pDst, pSrc, new IntPtr(count)); + fixed (byte* pDst = &dst[dstOffset]) + fixed (byte* pSrc = &src[srcOffset]) + { + NativeMethods.memmove(pDst, pSrc, new IntPtr(count)); + } } - #if MEASURE sw.Stop(); if (n < SliceHelpers.CopyDurations.Length) SliceHelpers.CopyDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); @@ -324,11 +340,13 @@ public static unsafe void CopyBytesUnsafe([NotNull] byte[] dst, int dstOffset, b var sw = System.Diagnostics.Stopwatch.StartNew(); #endif - fixed (byte* pDst = &dst[dstOffset]) + if (count > 0) { - NativeMethods.memmove(pDst, src, new IntPtr(count)); + fixed (byte* pDst = &dst[dstOffset]) + { + NativeMethods.memmove(pDst, src, new IntPtr(count)); + } } - #if MEASURE sw.Stop(); if (n < SliceHelpers.CopyDurations.Length) SliceHelpers.CopyDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); @@ -340,11 +358,14 @@ public static void SetBytes(byte[] bytes, int offset, int count, byte value) { SliceHelpers.EnsureBufferIsValid(bytes, offset, count); - unsafe + if (count > 0) { - fixed (byte* ptr = &bytes[offset]) + unsafe { - NativeMethods.memset(ptr, value, new IntPtr(count)); + fixed (byte* ptr = &bytes[offset]) + { + NativeMethods.memset(ptr, value, new IntPtr(count)); + } } } } From 27eade2bc2b00328b89fd87f8001c16c63209784 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Feb 2015 17:32:35 +0100 Subject: [PATCH 12/13] Slice: made sure that the x64 code generated by the JIT is the fastest possible for byte copy and compare: - The x64 assembly generated to get the addess of &left[offset] is smaller and faster, than getting the addess of left and adding the offset manually - &left[offset] also does a nullcheck and boundcheck at runtime "for free" --- FoundationDB.Client/Utils/SliceHelpers.cs | 47 +++++++++++------------ 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/FoundationDB.Client/Utils/SliceHelpers.cs b/FoundationDB.Client/Utils/SliceHelpers.cs index 6d4446e6b..7818b206f 100644 --- a/FoundationDB.Client/Utils/SliceHelpers.cs +++ b/FoundationDB.Client/Utils/SliceHelpers.cs @@ -220,28 +220,25 @@ public static bool SameBytesUnsafe([NotNull] byte[] left, int leftOffset, [NotNu int n = count; if (n < SliceHelpers.CompareHistogram.Length) ++SliceHelpers.CompareHistogram[n]; var sw = System.Diagnostics.Stopwatch.StartNew(); + try { #endif - int c; - if (count > 0) + if (count == 0) return true; + unsafe { - unsafe + fixed (byte* pLeft = &left[leftOffset]) + fixed (byte* pRight = &right[rightOffset]) { - fixed (byte* pLeft = &left[leftOffset]) - fixed (byte* pRight = &right[rightOffset]) - { - c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); - } + return 0 == NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); } } - else +#if MEASURE + } + finally { - c = 0; + sw.Stop(); + if (n < SliceHelpers.CompareDurations.Length) SliceHelpers.CompareDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); } -#if MEASURE - sw.Stop(); - if (n < SliceHelpers.CompareDurations.Length) SliceHelpers.CompareDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); #endif - return c == 0; } /// Compare two byte segments lexicographically, without validating the arguments @@ -265,28 +262,29 @@ public static int CompareBytesUnsafe([NotNull] byte[] left, int leftOffset, int int n = count; if (n < SliceHelpers.CompareHistogram.Length) ++SliceHelpers.CompareHistogram[n]; var sw = System.Diagnostics.Stopwatch.StartNew(); + try { #endif - int c; - if (count == 0) - { - c = 0; - } - else + if (count > 0) { unsafe { fixed (byte* pLeft = &left[leftOffset]) fixed (byte* pRight = &right[rightOffset]) { - c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); + int c = NativeMethods.memcmp(pLeft, pRight, new IntPtr(count)); + if (c != 0) return c; } } } + return leftCount - rightCount; #if MEASURE - sw.Stop(); - if (n < SliceHelpers.CompareDurations.Length) SliceHelpers.CompareDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); + } + finally + { + sw.Stop(); + if (n < SliceHelpers.CompareDurations.Length) SliceHelpers.CompareDurations[n] += (sw.Elapsed.TotalMilliseconds * 1E6); + } #endif - return c != 0 ? c : leftCount - rightCount; } #if MEASURE @@ -339,7 +337,6 @@ public static unsafe void CopyBytesUnsafe([NotNull] byte[] dst, int dstOffset, b if (n < SliceHelpers.CopyHistogram.Length) ++SliceHelpers.CopyHistogram[n]; var sw = System.Diagnostics.Stopwatch.StartNew(); #endif - if (count > 0) { fixed (byte* pDst = &dst[dstOffset]) From 580ea330cd22b06cb5b638159c8c3370c2cd8412 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Feb 2015 17:34:26 +0100 Subject: [PATCH 13/13] Slice: using a singleton for Encoding.UTF8 + use direct string ctors when possible - Encoding.UTF8 returns a new object everytime it is called, so cache one and reuse it - Use new string(sbyte*, ..., [Encoding]) instead of [Encoding].GetString(byte[], .....) whenever possible --- .../Layers/Tuples/FdbTupleParser.cs | 27 +++++------ FoundationDB.Client/Utils/Slice.cs | 45 +++++++++++++------ FoundationDB.Client/Utils/TinyJsonParser.cs | 4 +- 3 files changed, 45 insertions(+), 31 deletions(-) diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs index 15baae74c..d012039cb 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs @@ -365,7 +365,7 @@ public static unsafe void WriteString(ref TupleWriter writer, string value) { if (!TryWriteUnescapedUtf8String(ref writer, chars, value.Length)) { // the string contains \0 chars, we need to do it the hard way - WriteNulEscapedBytes(ref writer, FdbTupleTypes.Utf8, Encoding.UTF8.GetBytes(value)); + WriteNulEscapedBytes(ref writer, FdbTupleTypes.Utf8, Slice.Utf8Encoding.GetBytes(value)); } } } @@ -393,7 +393,7 @@ internal static unsafe void WriteChars(ref TupleWriter writer, char[] value, int { if (!TryWriteUnescapedUtf8String(ref writer, chars + offset, count)) { // the string contains \0 chars, we need to do it the hard way - WriteNulEscapedBytes(ref writer, FdbTupleTypes.Utf8, Encoding.UTF8.GetBytes(value, 0, count)); + WriteNulEscapedBytes(ref writer, FdbTupleTypes.Utf8, Slice.Utf8Encoding.GetBytes(value, 0, count)); } } } @@ -471,7 +471,7 @@ private static unsafe bool TryWriteUnescapedUtf8String(ref TupleWriter writer, c // > For small strings, we will allocated exactly string.Length * 3 bytes, and will be done in one chunk // > For larger strings, we will call encoder.Convert(...) until it says it is done. const int CHUNK_SIZE = 1024; - int bufLen = Encoding.UTF8.GetMaxByteCount(Math.Min(count, CHUNK_SIZE)); + int bufLen = Slice.Utf8Encoding.GetMaxByteCount(Math.Min(count, CHUNK_SIZE)); byte* buf = stackalloc byte[bufLen]; // We can not really predict the final size of the encoded string, but: @@ -480,7 +480,7 @@ private static unsafe bool TryWriteUnescapedUtf8String(ref TupleWriter writer, c writer.Output.EnsureBytes(checked(2 + count + (count >> 1))); // preallocate 150% of the string + 2 bytes writer.Output.UnsafeWriteByte(FdbTupleTypes.Utf8); - var encoder = Encoding.UTF8.GetEncoder(); + var encoder = Slice.Utf8Encoding.GetEncoder(); // note: encoder.Convert() tries to fill up the buffer as much as possible with complete chars, and will set 'done' to true when all chars have been converted. do { @@ -524,7 +524,7 @@ public static void WriteChar(ref TupleWriter writer, char value) { // 0x800..0xFFFF => 11110xxx 10xxxxxx 10xxxxxx 10xxxxxx // note: System.Char is 16 bits, and thus cannot represent UNICODE chars above 0xFFFF. // => This means that a System.Char will never take more than 3 bytes in UTF-8 ! - var tmp = Encoding.UTF8.GetBytes(new string(value, 1)); + var tmp = Slice.Utf8Encoding.GetBytes(new string(value, 1)); writer.Output.EnsureBytes(tmp.Length + 2); writer.Output.UnsafeWriteByte(FdbTupleTypes.Utf8); writer.Output.UnsafeWriteBytes(tmp, 0, tmp.Length); @@ -702,7 +702,7 @@ public static long ParseInt64(int type, Slice slice) return value; } - internal static ArraySegment UnescapeByteString([NotNull] byte[] buffer, int offset, int count) + internal static Slice UnescapeByteString([NotNull] byte[] buffer, int offset, int count) { Contract.Requires(buffer != null && offset >= 0 && count >= 0); @@ -719,10 +719,10 @@ internal static ArraySegment UnescapeByteString([NotNull] byte[] buffer, i ++p; } // buffer is clean, we can return it as-is - return new ArraySegment(buffer, offset, count); + return new Slice(buffer, offset, count); } - internal static ArraySegment UnescapeByteStringSlow([NotNull] byte[] buffer, int offset, int count, int offsetOfFirstZero = 0) + internal static Slice UnescapeByteStringSlow([NotNull] byte[] buffer, int offset, int count, int offsetOfFirstZero = 0) { Contract.Requires(buffer != null && offset >= 0 && count >= 0); @@ -749,7 +749,7 @@ internal static ArraySegment UnescapeByteStringSlow([NotNull] byte[] buffe tmp[i++] = b; } - return new ArraySegment(tmp, 0, i); + return new Slice(tmp, 0, i); } /// Parse a tuple segment containing a byte array @@ -758,9 +758,7 @@ public static Slice ParseBytes(Slice slice) Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Bytes && slice[-1] == 0); if (slice.Count <= 2) return Slice.Empty; - var decoded = UnescapeByteString(slice.Array, slice.Offset + 1, slice.Count - 2); - - return new Slice(decoded.Array, decoded.Offset, decoded.Count); + return UnescapeByteString(slice.Array, slice.Offset + 1, slice.Count - 2); } /// Parse a tuple segment containing an ASCII string stored as a byte array @@ -771,8 +769,7 @@ public static string ParseAscii(Slice slice) if (slice.Count <= 2) return String.Empty; var decoded = UnescapeByteString(slice.Array, slice.Offset + 1, slice.Count - 2); - - return Slice.DefaultEncoding.GetString(decoded.Array, decoded.Offset, decoded.Count); + return decoded.ToAscii(); } /// Parse a tuple segment containing a unicode string @@ -783,7 +780,7 @@ public static string ParseUnicode(Slice slice) if (slice.Count <= 2) return String.Empty; //TODO: check args var decoded = UnescapeByteString(slice.Array, slice.Offset + 1, slice.Count - 2); - return Encoding.UTF8.GetString(decoded.Array, decoded.Offset, decoded.Count); + return decoded.ToUnicode(); } /// Parse a tuple segment containing an embedded tuple diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs index 9b910dfd2..b56b42f9c 100644 --- a/FoundationDB.Client/Utils/Slice.cs +++ b/FoundationDB.Client/Utils/Slice.cs @@ -830,14 +830,15 @@ public static Slice FromUuid64(Uuid64 value) return value.ToSlice(); } - internal static readonly Encoding DefaultEncoding = #if CORE_CLR - Encoding.GetEncoding(0); + internal static readonly Encoding DefaultEncoding = Encoding.GetEncoding(0); #else - Encoding.Default; + internal static readonly Encoding DefaultEncoding = Encoding.Default; #endif - /// Dangerously create a slice containing string converted to ASCII. All non-ASCII characters may be corrupted or converted to '?' + internal static readonly Encoding Utf8Encoding = new UTF8Encoding(false); + + /// Dangerously create a slice containing string converted to ASCII, using the system default codepage. All non-ASCII characters may be corrupted or converted to '?' /// WARNING: if you put a string that contains non-ASCII chars, it will be silently corrupted! This should only be used to store keywords or 'safe' strings. /// Note: depending on your default codepage, chars from 128 to 255 may be preserved, but only if they are decoded using the same codepage at the other end ! public static Slice FromAscii(string text) @@ -848,11 +849,11 @@ public static Slice FromAscii(string text) /// Create a slice containing the UTF-8 bytes of the string public static Slice FromString(string value) { - return value == null ? Slice.Nil : value.Length == 0 ? Slice.Empty : Slice.Create(Encoding.UTF8.GetBytes(value)); + return value == null ? Slice.Nil : value.Length == 0 ? Slice.Empty : Slice.Create(Utf8Encoding.GetBytes(value)); } /// Create a slice that holds the UTF-8 encoded representation of - /// + /// Unicode character /// The returned slice is only guaranteed to hold 1 byte for ASCII chars (0..127). For non-ASCII chars, the size can be from 1 to 6 bytes. /// If you need to use ASCII chars, you should use Slice.FromByte() instead public static Slice FromChar(char value) @@ -864,7 +865,7 @@ public static Slice FromChar(char value) // note: Encoding.UTF8.GetMaxByteCount(1) returns 6, but allocate 8 to stay aligned var tmp = new byte[8]; - int n = Encoding.UTF8.GetBytes(new char[] { value }, 0, 1, tmp, 0); + int n = Slice.Utf8Encoding.GetBytes(new char[] { value }, 0, 1, tmp, 0); return n == 1 ? FromByte(tmp[0]) : new Slice(tmp, 0, n); } @@ -969,9 +970,17 @@ public SliceStream AsStream() [Pure, CanBeNull] public string ToAscii() { - if (this.Count == 0) return this.HasValue ? String.Empty : default(string); SliceHelpers.EnsureSliceIsValid(ref this); - return Slice.DefaultEncoding.GetString(this.Array, this.Offset, this.Count); + + if (this.Count == 0) return this.HasValue ? String.Empty : default(string); + unsafe + { + fixed (byte* ptr = this.Array) + { + return new string((sbyte*)ptr, this.Offset, this.Count); + } + } + //return Slice.DefaultEncoding.GetString(this.Array, this.Offset, this.Count); } /// Stringify a slice containing an UTF-8 encoded string @@ -979,9 +988,17 @@ public string ToAscii() [Pure, CanBeNull] public string ToUnicode() { - if (this.Count == 0) return this.HasValue ? String.Empty : default(string); SliceHelpers.EnsureSliceIsValid(ref this); - return Encoding.UTF8.GetString(this.Array, this.Offset, this.Count); + + if (this.Count == 0) return this.HasValue ? String.Empty : default(string); + unsafe + { + fixed (byte* ptr = this.Array) + { + return new string((sbyte*)ptr, this.Offset, this.Count, Utf8Encoding); + } + } + //return Slice.Utf8Encoding.GetString(this.Array, this.Offset, this.Count); } /// Converts a slice using Base64 encoding @@ -1073,7 +1090,7 @@ public string ToAsciiOrHexaString() //REVIEW: rename this to ToPrintableString() // look for UTF-8 BOM if (n >= 3 && buffer[p] == 0xEF && buffer[p + 1] == 0xBB && buffer[p + 2] == 0xBF) { // this is supposed to be an UTF-8 string - return EscapeString(new StringBuilder(n).Append('\''), buffer, p + 3, n - 3, Encoding.UTF8).Append('\'').ToString(); + return EscapeString(new StringBuilder(n).Append('\''), buffer, p + 3, n - 3, Slice.Utf8Encoding).Append('\'').ToString(); } if (n >= 2) @@ -1081,7 +1098,7 @@ public string ToAsciiOrHexaString() //REVIEW: rename this to ToPrintableString() // look for JSON objets or arrays if ((buffer[p] == '{' && buffer[p + n - 1] == '}') || (buffer[p] == '[' && buffer[p + n - 1] == ']')) { - return EscapeString(new StringBuilder(n + 16), buffer, p, n, Encoding.UTF8).ToString(); + return EscapeString(new StringBuilder(n + 16), buffer, p, n, Slice.Utf8Encoding).ToString(); } } @@ -1111,7 +1128,7 @@ public string ToAsciiOrHexaString() //REVIEW: rename this to ToPrintableString() } else { // some escaping required - return EscapeString(new StringBuilder(n + 2).Append('\''), buffer, this.Offset, this.Count, Encoding.UTF8).Append('\'').ToString(); + return EscapeString(new StringBuilder(n + 2).Append('\''), buffer, this.Offset, this.Count, Slice.Utf8Encoding).Append('\'').ToString(); } } diff --git a/FoundationDB.Client/Utils/TinyJsonParser.cs b/FoundationDB.Client/Utils/TinyJsonParser.cs index 727d42c80..c32c2e5db 100644 --- a/FoundationDB.Client/Utils/TinyJsonParser.cs +++ b/FoundationDB.Client/Utils/TinyJsonParser.cs @@ -321,7 +321,7 @@ private FormatException SyntaxError(string msg, object arg0) public static Dictionary ParseObject(Slice data) { if (data.Count == 0) return null; - char[] chars = Encoding.UTF8.GetChars(data.Array, data.Offset, data.Count); + char[] chars = Slice.Utf8Encoding.GetChars(data.Array, data.Offset, data.Count); return ParseObject(chars, 0, chars.Length); } @@ -356,7 +356,7 @@ internal static Dictionary ParseObject([NotNull] char[] chars, i public static List ParseArray(Slice data) { if (data.Count == 0) return null; - char[] chars = Encoding.UTF8.GetChars(data.Array, data.Offset, data.Count); + char[] chars = Slice.Utf8Encoding.GetChars(data.Array, data.Offset, data.Count); var parser = new TinyJsonParser(chars, 0, chars.Length); var token = parser.ReadToken(); if (token == Token.Eof) return null;